Merge fullstack_asterix_stabilization into fullstack_hyracks_result_distribution branch.

git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_hyracks_result_distribution@2862 123451ca-8445-de46-9d55-352943316053
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index f1e7acb..dde4443 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
@@ -44,6 +45,7 @@
     protected ITypeTraitProvider typeTraitProvider;
     protected ISerializerDeserializerProvider serializerDeserializerProvider;
     protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
+    protected IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
     protected IBinaryComparatorFactoryProvider comparatorFactoryProvider;
     protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
     protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
@@ -94,6 +96,14 @@
         return hashFunctionFactoryProvider;
     }
 
+    public void setHashFunctionFamilyProvider(IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider) {
+        this.hashFunctionFamilyProvider = hashFunctionFamilyProvider;
+    }
+
+    public IBinaryHashFunctionFamilyProvider getHashFunctionFamilyProvider() {
+        return hashFunctionFamilyProvider;
+    }
+
     public void setComparatorFactoryProvider(IBinaryComparatorFactoryProvider comparatorFactoryProvider) {
         this.comparatorFactoryProvider = comparatorFactoryProvider;
     }
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 1d21463..edc1b66 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -84,11 +84,12 @@
                     public JobSpecification createJob(Object appContext) throws AlgebricksException {
                         AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Starting Job Generation.\n");
                         JobGenContext context = new JobGenContext(null, metadata, appContext,
-                                serializerDeserializerProvider, hashFunctionFactoryProvider, comparatorFactoryProvider,
-                                typeTraitProvider, binaryBooleanInspectorFactory, binaryIntegerInspectorFactory,
-                                printerProvider, nullWriterFactory, normalizedKeyComputerFactoryProvider,
-                                expressionRuntimeProvider, expressionTypeComputer, nullableTypeComputer, oc,
-                                expressionEvalSizeComputer, partialAggregationTypeComputer, frameSize, clusterLocations);
+                                serializerDeserializerProvider, hashFunctionFactoryProvider,
+                                hashFunctionFamilyProvider, comparatorFactoryProvider, typeTraitProvider,
+                                binaryBooleanInspectorFactory, binaryIntegerInspectorFactory, printerProvider,
+                                nullWriterFactory, normalizedKeyComputerFactoryProvider, expressionRuntimeProvider,
+                                expressionTypeComputer, nullableTypeComputer, oc, expressionEvalSizeComputer,
+                                partialAggregationTypeComputer, frameSize, clusterLocations);
                         PlanCompiler pc = new PlanCompiler(context);
                         return pc.compilePlan(plan, null);
                     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
index 165fccd..6701385 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
@@ -89,4 +89,9 @@
     public IPhysicalPropertiesVector getDeliveredPhysicalProperties();
 
     public void computeDeliveredPhysicalProperties(IOptimizationContext context) throws AlgebricksException;
+    
+    /**
+     * Indicates whether the expressions used by this operator must be variable reference expressions.
+     */
+    public boolean requiresVariableReferenceExpressions();
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
index 468d25c..0aa1ff6 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -51,6 +51,8 @@
      * returns true if op1 and op2 have already been compared
      */
     public abstract boolean checkAndAddToAlreadyCompared(ILogicalOperator op1, ILogicalOperator op2);
+    
+    public abstract void removeFromAlreadyCompared(ILogicalOperator op1);
 
     public abstract void addNotToBeInlinedVar(LogicalVariable var);
 
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
index bf3f82b..b284b22 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
@@ -39,9 +39,11 @@
 
     @Override
     public ScalarFunctionCallExpression cloneExpression() {
-        cloneAnnotations();
         List<Mutable<ILogicalExpression>> clonedArgs = cloneArguments();
-        return new ScalarFunctionCallExpression(finfo, clonedArgs);
+        ScalarFunctionCallExpression funcExpr = new ScalarFunctionCallExpression(finfo, clonedArgs);
+        funcExpr.getAnnotations().putAll(cloneAnnotations());
+        funcExpr.setOpaqueParameters(this.getOpaqueParameters());
+        return funcExpr;
     }
 
     @Override
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
index 71932d8..652f9b0 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
@@ -45,6 +45,7 @@
         List<Mutable<ILogicalExpression>> clonedArgs = cloneArguments();
         UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(finfo, clonedArgs);
         ufce.setReturnsUniqueValues(returnsUniqueValues);
+        ufce.setOpaqueParameters(this.getOpaqueParameters());
         return ufce;
     }
 
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
index dc0edfe..64dbdef 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -182,4 +182,9 @@
         return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getNullableTypeComputer(),
                 ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
     }
+    
+    @Override
+    public boolean requiresVariableReferenceExpressions() {
+        return true;
+    }
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
index a4dc2e0..4543997 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
@@ -89,4 +89,8 @@
         return env;
     }
 
+    @Override
+    public boolean requiresVariableReferenceExpressions() {
+        return false;
+    }
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index 3227f3d..3c21c8f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -106,5 +106,4 @@
         }
         return env;
     }
-
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java
index 20aa574..03bfcba 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java
@@ -81,4 +81,8 @@
         return createPropagatingAllInputsTypeEnvironment(ctx);
     }
 
+    @Override
+    public boolean requiresVariableReferenceExpressions() {
+        return false;
+    }
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
index ee0dcf6..b1da831 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;

 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;

 

@@ -49,12 +50,34 @@
 

     @Override

     public void recomputeSchema() {

-        schema = new ArrayList<LogicalVariable>(inputs.get(0).getValue().getSchema());

+        if (schema == null) {

+            schema = new ArrayList<LogicalVariable>();

+        }

+        schema.clear();

+        for (Mutable<ILogicalExpression> eRef : expressions) {

+            ILogicalExpression e = eRef.getValue();

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

+                VariableReferenceExpression v = (VariableReferenceExpression) e;

+                schema.add(v.getVariableReference());

+            }

+        }

     }

 

     @Override

     public VariablePropagationPolicy getVariablePropagationPolicy() {

-        return VariablePropagationPolicy.ALL;

+        return new VariablePropagationPolicy() {

+            @Override

+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)

+                    throws AlgebricksException {

+                for (Mutable<ILogicalExpression> eRef : expressions) {

+                    ILogicalExpression e = eRef.getValue();

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

+                        VariableReferenceExpression v = (VariableReferenceExpression) e;

+                        target.addVariable(v.getVariableReference());

+                    }

+                }

+            }

+        };

     }

 

     @Override

@@ -105,7 +128,16 @@
 

     @Override

     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {

-        return createPropagatingAllInputsTypeEnvironment(ctx);

+        IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());

+        IVariableTypeEnvironment childEnv = ctx.getOutputTypeEnvironment(inputs.get(0).getValue());

+        for (Mutable<ILogicalExpression> exprRef : expressions) {

+            ILogicalExpression expr = exprRef.getValue();

+            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

+                VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr;

+                env.setVarType(varRefExpr.getVariableReference(), childEnv.getType(expr));

+            }

+        }

+        return env;

     }

 

 }

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExtensionOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExtensionOperator.java
index 101b6f5..5aa858f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExtensionOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExtensionOperator.java
@@ -51,7 +51,7 @@
 
     @Override
     public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
-        return false;
+        return delegate.acceptExpressionTransform(transform);
     }
 
     @Override
@@ -108,4 +108,13 @@
         return this.createPropagatingAllInputsTypeEnvironment(ctx);
     }
 
+    @Override
+    public String toString() {
+        return delegate.toString();
+    }
+    
+    public IOperatorExtension getDelegate() {
+        return delegate;
+    }
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorExtension.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorExtension.java
index 98c3301..0a80337 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorExtension.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorExtension.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
 
+import java.util.Collection;
 import java.util.List;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -42,5 +43,8 @@
     void setPhysicalOperator(IPhysicalOperator physicalOperator);
 
     ExecutionMode getExecutionMode();
-
+    
+    public void getUsedVariables(Collection<LogicalVariable> usedVars);
+    
+    public void getProducedVariables(Collection<LogicalVariable> producedVars);
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java
index 3c6a699..8e67ed9 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java
@@ -109,5 +109,4 @@
     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
         return createPropagatingAllInputsTypeEnvironment(ctx);
     }
-
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index 8c611b0..fda920a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -102,4 +102,8 @@
         return env;
     }
 
+    @Override
+    public boolean requiresVariableReferenceExpressions() {
+        return false;
+    }
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 31061db..b97597d 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
@@ -345,7 +345,7 @@
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
         if (aop.getOperatorTag() != LogicalOperatorTag.UNNEST_MAP)
             return Boolean.FALSE;
-        UnnestOperator unnestOpArg = (UnnestOperator) copyAndSubstituteVar(op, arg);
+        UnnestMapOperator unnestOpArg = (UnnestMapOperator) copyAndSubstituteVar(op, arg);
         boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), unnestOpArg.getVariables());
         if (!isomorphic)
             return Boolean.FALSE;
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 994c6cb..78b6801 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
@@ -249,6 +249,7 @@
 

     @Override

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

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

         return null;

     }

 }

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 9295179..a759e35 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
@@ -85,7 +85,13 @@
 

     @Override

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

-        standardLayout(op);

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

+            ILogicalExpression expr = exprRef.getValue();

+            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

+                VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr;

+                schemaVariables.add(varRefExpr.getVariableReference());

+            }

+        }

         return null;

     }

 

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 3a82ccd..0ea9367 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
@@ -25,6 +25,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;

 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;

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

+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;

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

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

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

@@ -33,6 +34,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

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

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

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

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

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

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

@@ -49,13 +51,17 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

 

 public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {

@@ -106,8 +112,49 @@
     }

 

     @Override

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

-        // does not use any variable

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

+        // Used variables depend on the physical operator.

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

+            IPhysicalOperator physOp = op.getPhysicalOperator();

+            switch (physOp.getOperatorTag()) {

+                case BROADCAST_EXCHANGE:

+                case ONE_TO_ONE_EXCHANGE:

+                case RANDOM_MERGE_EXCHANGE: {

+                    // No variables used.

+                    break;

+                }

+                case HASH_PARTITION_EXCHANGE: {

+                    HashPartitionExchangePOperator concreteOp = (HashPartitionExchangePOperator) physOp;

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

+                    break;

+                }

+                case HASH_PARTITION_MERGE_EXCHANGE: {

+                    HashPartitionMergeExchangePOperator concreteOp = (HashPartitionMergeExchangePOperator) physOp;

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

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

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

+                    }

+                    break;

+                }

+                case SORT_MERGE_EXCHANGE: {

+                    SortMergeExchangePOperator concreteOp = (SortMergeExchangePOperator) physOp;

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

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

+                    }

+                    break;

+                }

+                case RANGE_PARTITION_EXCHANGE: {

+                    RangePartitionPOperator concreteOp = (RangePartitionPOperator) physOp;

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

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

+                    }

+                    break;

+                }

+                default: {

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

+                }

+            }

+        }

         return null;

     }

 

@@ -297,6 +344,7 @@
 

     @Override

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

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

         return null;

     }

 

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
index 25f22e7..f66f99b 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
@@ -61,14 +61,22 @@
             ITypingContext ctx) throws AlgebricksException {

         substituteVariables(op, v1, v2, true, ctx);

     }

-

+    

+    public static void substituteVariablesInDescendantsAndSelf(ILogicalOperator op, LogicalVariable v1,

+            LogicalVariable v2, ITypingContext ctx) throws AlgebricksException {

+        for (Mutable<ILogicalOperator> childOp : op.getInputs()) {

+            substituteVariablesInDescendantsAndSelf(childOp.getValue(), v1, v2, ctx);

+        }

+        substituteVariables(op, v1, v2, true, ctx);

+    }

+    

     public static void substituteVariables(ILogicalOperator op, LogicalVariable v1, LogicalVariable v2,

             boolean goThroughNts, ITypingContext ctx) throws AlgebricksException {

         ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> visitor = new SubstituteVariableVisitor(

                 goThroughNts, ctx);

         op.accept(visitor, new Pair<LogicalVariable, LogicalVariable>(v1, v2));

     }

-

+    

     public static <T> boolean varListEqualUnordered(List<T> var, List<T> varArg) {

         Set<T> varSet = new HashSet<T>();

         Set<T> varArgSet = new HashSet<T>();

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
index f3c9e5a..61d4880 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
@@ -158,5 +158,13 @@
                 comparatorFactories);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
     }
+    
+    public List<LogicalVariable> getPartitionFields() {
+        return partitionFields;
+    }
+    
+    public List<OrderColumn> getOrderColumns() {
+        return orderColumns;
+    }
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index c737cc4..6da42b4 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -32,14 +32,21 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.join.OptimizedHybridHashJoinOperatorDescriptor;
 
 public class HybridHashJoinPOperator extends AbstractHashJoinPOperator {
 
@@ -90,6 +97,8 @@
         IVariableTypeEnvironment env = context.getTypeEnvironment(op);
         IBinaryHashFunctionFactory[] hashFunFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
                 keysLeftBranch, env, context);
+        IBinaryHashFunctionFamily[] hashFunFamilies = JobGenHelper.variablesToBinaryHashFunctionFamilies(
+                keysLeftBranch, env, context);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[keysLeft.length];
         int i = 0;
         IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
@@ -97,33 +106,75 @@
             Object t = env.getVarType(v);
             comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
         }
-        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IOperatorDescriptor opDesc = null;
-        try {
-            switch (kind) {
-                case INNER: {
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
-                            hashFunFactories, comparatorFactories, recDescriptor);
-                    break;
-                }
-                case LEFT_OUTER: {
-                    INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
-                    for (int j = 0; j < nullWriterFactories.length; j++) {
-                        nullWriterFactories[j] = context.getNullWriterFactory();
-                    }
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
-                            hashFunFactories, comparatorFactories, recDescriptor, true, nullWriterFactories);
-                    break;
-                }
-                default: {
-                    throw new NotImplementedException();
-                }
+
+        boolean optimizedHashJoin = true;
+        for (IBinaryHashFunctionFamily family : hashFunFamilies) {
+            if (family == null) {
+                optimizedHashJoin = false;
+                break;
             }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        }
+
+        if (!optimizedHashJoin) {
+            try {
+                switch (kind) {
+                    case INNER: {
+                        opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                                maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
+                                hashFunFactories, comparatorFactories, recDescriptor);
+                        break;
+                    }
+                    case LEFT_OUTER: {
+                        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
+                        for (int j = 0; j < nullWriterFactories.length; j++) {
+                            nullWriterFactories[j] = context.getNullWriterFactory();
+                        }
+                        opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                                maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
+                                hashFunFactories, comparatorFactories, recDescriptor, true, nullWriterFactories);
+                        break;
+                    }
+                    default: {
+                        throw new NotImplementedException();
+                    }
+                }
+            } catch (HyracksDataException e) {
+                throw new AlgebricksException(e);
+            }
+        } else {
+            try {
+                switch (kind) {
+                    case INNER: {
+                        opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                                maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                                comparatorFactories, recDescriptor, new JoinMultiComparatorFactory(comparatorFactories,
+                                        keysLeft, keysRight), new JoinMultiComparatorFactory(comparatorFactories,
+                                        keysRight, keysLeft));
+                        break;
+                    }
+                    case LEFT_OUTER: {
+                        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
+                        for (int j = 0; j < nullWriterFactories.length; j++) {
+                            nullWriterFactories[j] = context.getNullWriterFactory();
+                        }
+                        opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                                maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                                comparatorFactories, recDescriptor, new JoinMultiComparatorFactory(comparatorFactories,
+                                        keysLeft, keysRight), new JoinMultiComparatorFactory(comparatorFactories,
+                                        keysRight, keysLeft), true, nullWriterFactories);
+                        break;
+                    }
+                    default: {
+                        throw new NotImplementedException();
+                    }
+                }
+            } catch (HyracksDataException e) {
+                throw new AlgebricksException(e);
+            }
         }
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
@@ -140,3 +191,72 @@
     }
 
 }
+
+/**
+ * {@ ITuplePairComparatorFactory} implementation for optimized hybrid hash join.
+ */
+class JoinMultiComparatorFactory implements ITuplePairComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IBinaryComparatorFactory[] binaryComparatorFactories;
+    private final int[] keysLeft;
+    private final int[] keysRight;
+
+    public JoinMultiComparatorFactory(IBinaryComparatorFactory[] binaryComparatorFactory, int[] keysLeft,
+            int[] keysRight) {
+        this.binaryComparatorFactories = binaryComparatorFactory;
+        this.keysLeft = keysLeft;
+        this.keysRight = keysRight;
+    }
+
+    @Override
+    public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) {
+        IBinaryComparator[] binaryComparators = new IBinaryComparator[binaryComparatorFactories.length];
+        for (int i = 0; i < binaryComparators.length; i++) {
+            binaryComparators[i] = binaryComparatorFactories[i].createBinaryComparator();
+        }
+        return new JoinMultiComparator(binaryComparators, keysLeft, keysRight);
+    }
+}
+
+/**
+ * {@ ITuplePairComparator} implementation for optimized hybrid hash join.
+ * The comparator applies multiple binary comparators, one for each key pairs
+ */
+class JoinMultiComparator implements ITuplePairComparator {
+    private final IBinaryComparator[] binaryComparators;
+    private final int[] keysLeft;
+    private final int[] keysRight;
+
+    public JoinMultiComparator(IBinaryComparator[] bComparator, int[] keysLeft, int[] keysRight) {
+        this.binaryComparators = bComparator;
+        this.keysLeft = keysLeft;
+        this.keysRight = keysRight;
+    }
+
+    @Override
+    public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1) {
+        int tStart0 = accessor0.getTupleStartOffset(tIndex0);
+        int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
+
+        int tStart1 = accessor1.getTupleStartOffset(tIndex1);
+        int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
+
+        for (int i = 0; i < binaryComparators.length; i++) {
+            int fStart0 = accessor0.getFieldStartOffset(tIndex0, keysLeft[i]);
+            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, keysLeft[i]);
+            int fLen0 = fEnd0 - fStart0;
+
+            int fStart1 = accessor1.getFieldStartOffset(tIndex1, keysRight[i]);
+            int fEnd1 = accessor1.getFieldEndOffset(tIndex1, keysRight[i]);
+            int fLen1 = fEnd1 - fStart1;
+
+            int c = binaryComparators[i].compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0,
+                    accessor1.getBuffer().array(), fStart1 + fStartOffset1, fLen1);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
index 8cbd2d8..d153f90 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
@@ -44,6 +44,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -56,8 +57,8 @@
 import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
 
 /**
- * Left input is broadcast and preserves its local properties.
- * Right input can be partitioned in any way.
+ * Left input is broadcast and preserves its local properties. Right input can
+ * be partitioned in any way.
  */
 public class NLJoinPOperator extends AbstractJoinPOperator {
 
@@ -97,7 +98,7 @@
                 pp = pv1.getPartitioningProperty();
             }
         } else {
-        	pp = IPartitioningProperty.UNPARTITIONED;
+            pp = IPartitioningProperty.UNPARTITIONED;
         }
 
         List<ILocalStructuralProperty> localProps = new LinkedList<ILocalStructuralProperty>();
@@ -122,7 +123,8 @@
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
-        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                propagatedSchema, context);
         IOperatorSchema[] conditionInputSchemas = new IOperatorSchema[1];
         conditionInputSchemas[0] = propagatedSchema;
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
@@ -135,10 +137,19 @@
 
         switch (kind) {
             case INNER: {
-                opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize);
+                opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, false,
+                        null);
                 break;
             }
-            case LEFT_OUTER:
+            case LEFT_OUTER: {
+                INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
+                for (int j = 0; j < nullWriterFactories.length; j++) {
+                    nullWriterFactories[j] = context.getNullWriterFactory();
+                }
+                opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
+                        nullWriterFactories);
+                break;
+            }
             default: {
                 throw new NotImplementedException();
             }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
index 7e3c935..8875f6c 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
@@ -16,6 +16,7 @@
 
 import java.util.ArrayList;
 import java.util.LinkedList;
+import java.util.List;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -69,5 +70,9 @@
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
         throw new NotImplementedException();
     }
+    
+    public List<OrderColumn> getPartitioningFields() {
+        return partitioningFields;
+    }
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 22a1a81..365d1a5 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
@@ -42,151 +43,167 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 
 public class JobGenContext {
-    private final IOperatorSchema outerFlowSchema;
-    private final Map<ILogicalOperator, IOperatorSchema> schemaMap = new HashMap<ILogicalOperator, IOperatorSchema>();
-    private final ISerializerDeserializerProvider serializerDeserializerProvider;
-    private final IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
-    private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
-    private final IPrinterFactoryProvider printerFactoryProvider;
-    private final ITypeTraitProvider typeTraitProvider;
-    private final IMetadataProvider<?, ?> metadataProvider;
-    private final INullWriterFactory nullWriterFactory;
-    private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
-    private final Object appContext;
-    private final IBinaryBooleanInspectorFactory booleanInspectorFactory;
-    private final IBinaryIntegerInspectorFactory integerInspectorFactory;
-    private final IExpressionRuntimeProvider expressionRuntimeProvider;
-    private final IExpressionTypeComputer expressionTypeComputer;
-    private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
-    private final IPartialAggregationTypeComputer partialAggregationTypeComputer;
-    private final int frameSize;
-    private AlgebricksPartitionConstraint clusterLocations;
-    private int varCounter;
-    private final ITypingContext typingContext;
+	private final IOperatorSchema outerFlowSchema;
+	private final Map<ILogicalOperator, IOperatorSchema> schemaMap = new HashMap<ILogicalOperator, IOperatorSchema>();
+	private final ISerializerDeserializerProvider serializerDeserializerProvider;
+	private final IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
+	private final IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
+	private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
+	private final IPrinterFactoryProvider printerFactoryProvider;
+	private final ITypeTraitProvider typeTraitProvider;
+	private final IMetadataProvider<?, ?> metadataProvider;
+	private final INullWriterFactory nullWriterFactory;
+	private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
+	private final Object appContext;
+	private final IBinaryBooleanInspectorFactory booleanInspectorFactory;
+	private final IBinaryIntegerInspectorFactory integerInspectorFactory;
+	private final IExpressionRuntimeProvider expressionRuntimeProvider;
+	private final IExpressionTypeComputer expressionTypeComputer;
+	private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
+	private final IPartialAggregationTypeComputer partialAggregationTypeComputer;
+	private final int frameSize;
+	private AlgebricksPartitionConstraint clusterLocations;
+	private int varCounter;
+	private final ITypingContext typingContext;
 
-    public JobGenContext(IOperatorSchema outerFlowSchema, IMetadataProvider<?, ?> metadataProvider, Object appContext,
-            ISerializerDeserializerProvider serializerDeserializerProvider,
-            IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider,
-            IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
-            IBinaryBooleanInspectorFactory booleanInspectorFactory,
-            IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
-            INullWriterFactory nullWriterFactory,
-            INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
-            IExpressionRuntimeProvider expressionRuntimeProvider, IExpressionTypeComputer expressionTypeComputer,
-            INullableTypeComputer nullableTypeComputer, ITypingContext typingContext,
-            IExpressionEvalSizeComputer expressionEvalSizeComputer,
-            IPartialAggregationTypeComputer partialAggregationTypeComputer, int frameSize,
-            AlgebricksPartitionConstraint clusterLocations) {
-        this.outerFlowSchema = outerFlowSchema;
-        this.metadataProvider = metadataProvider;
-        this.appContext = appContext;
-        this.serializerDeserializerProvider = serializerDeserializerProvider;
-        this.hashFunctionFactoryProvider = hashFunctionFactoryProvider;
-        this.comparatorFactoryProvider = comparatorFactoryProvider;
-        this.typeTraitProvider = typeTraitProvider;
-        this.booleanInspectorFactory = booleanInspectorFactory;
-        this.integerInspectorFactory = integerInspectorFactory;
-        this.printerFactoryProvider = printerFactoryProvider;
-        this.clusterLocations = clusterLocations;
-        this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
-        this.nullWriterFactory = nullWriterFactory;
-        this.expressionRuntimeProvider = expressionRuntimeProvider;
-        this.expressionTypeComputer = expressionTypeComputer;
-        this.typingContext = typingContext;
-        this.expressionEvalSizeComputer = expressionEvalSizeComputer;
-        this.partialAggregationTypeComputer = partialAggregationTypeComputer;
-        this.frameSize = frameSize;
-        this.varCounter = 0;
-    }
+	public JobGenContext(
+			IOperatorSchema outerFlowSchema,
+			IMetadataProvider<?, ?> metadataProvider,
+			Object appContext,
+			ISerializerDeserializerProvider serializerDeserializerProvider,
+			IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider,
+			IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider,
+			IBinaryComparatorFactoryProvider comparatorFactoryProvider,
+			ITypeTraitProvider typeTraitProvider,
+			IBinaryBooleanInspectorFactory booleanInspectorFactory,
+			IBinaryIntegerInspectorFactory integerInspectorFactory,
+			IPrinterFactoryProvider printerFactoryProvider,
+			INullWriterFactory nullWriterFactory,
+			INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
+			IExpressionRuntimeProvider expressionRuntimeProvider,
+			IExpressionTypeComputer expressionTypeComputer,
+			INullableTypeComputer nullableTypeComputer,
+			ITypingContext typingContext,
+			IExpressionEvalSizeComputer expressionEvalSizeComputer,
+			IPartialAggregationTypeComputer partialAggregationTypeComputer,
+			int frameSize, AlgebricksPartitionConstraint clusterLocations) {
+		this.outerFlowSchema = outerFlowSchema;
+		this.metadataProvider = metadataProvider;
+		this.appContext = appContext;
+		this.serializerDeserializerProvider = serializerDeserializerProvider;
+		this.hashFunctionFactoryProvider = hashFunctionFactoryProvider;
+		this.hashFunctionFamilyProvider = hashFunctionFamilyProvider;
+		this.comparatorFactoryProvider = comparatorFactoryProvider;
+		this.typeTraitProvider = typeTraitProvider;
+		this.booleanInspectorFactory = booleanInspectorFactory;
+		this.integerInspectorFactory = integerInspectorFactory;
+		this.printerFactoryProvider = printerFactoryProvider;
+		this.clusterLocations = clusterLocations;
+		this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
+		this.nullWriterFactory = nullWriterFactory;
+		this.expressionRuntimeProvider = expressionRuntimeProvider;
+		this.expressionTypeComputer = expressionTypeComputer;
+		this.typingContext = typingContext;
+		this.expressionEvalSizeComputer = expressionEvalSizeComputer;
+		this.partialAggregationTypeComputer = partialAggregationTypeComputer;
+		this.frameSize = frameSize;
+		this.varCounter = 0;
+	}
 
-    public IOperatorSchema getOuterFlowSchema() {
-        return outerFlowSchema;
-    }
+	public IOperatorSchema getOuterFlowSchema() {
+		return outerFlowSchema;
+	}
 
-    public AlgebricksPartitionConstraint getClusterLocations() {
-        return clusterLocations;
-    }
+	public AlgebricksPartitionConstraint getClusterLocations() {
+		return clusterLocations;
+	}
 
-    public IMetadataProvider<?, ?> getMetadataProvider() {
-        return metadataProvider;
-    }
+	public IMetadataProvider<?, ?> getMetadataProvider() {
+		return metadataProvider;
+	}
 
-    public Object getAppContext() {
-        return appContext;
-    }
+	public Object getAppContext() {
+		return appContext;
+	}
 
-    public ISerializerDeserializerProvider getSerializerDeserializerProvider() {
-        return serializerDeserializerProvider;
-    }
+	public ISerializerDeserializerProvider getSerializerDeserializerProvider() {
+		return serializerDeserializerProvider;
+	}
 
-    public IBinaryHashFunctionFactoryProvider getBinaryHashFunctionFactoryProvider() {
-        return hashFunctionFactoryProvider;
-    }
+	public IBinaryHashFunctionFactoryProvider getBinaryHashFunctionFactoryProvider() {
+		return hashFunctionFactoryProvider;
+	}
 
-    public IBinaryComparatorFactoryProvider getBinaryComparatorFactoryProvider() {
-        return comparatorFactoryProvider;
-    }
+	public IBinaryHashFunctionFamilyProvider getBinaryHashFunctionFamilyProvider() {
+		return hashFunctionFamilyProvider;
+	}
 
-    public ITypeTraitProvider getTypeTraitProvider() {
-        return typeTraitProvider;
-    }
+	public IBinaryComparatorFactoryProvider getBinaryComparatorFactoryProvider() {
+		return comparatorFactoryProvider;
+	}
 
-    public IBinaryBooleanInspectorFactory getBinaryBooleanInspectorFactory() {
-        return booleanInspectorFactory;
-    }
+	public ITypeTraitProvider getTypeTraitProvider() {
+		return typeTraitProvider;
+	}
 
-    public IBinaryIntegerInspectorFactory getBinaryIntegerInspectorFactory() {
-        return integerInspectorFactory;
-    }
+	public IBinaryBooleanInspectorFactory getBinaryBooleanInspectorFactory() {
+		return booleanInspectorFactory;
+	}
 
-    public IPrinterFactoryProvider getPrinterFactoryProvider() {
-        return printerFactoryProvider;
-    }
+	public IBinaryIntegerInspectorFactory getBinaryIntegerInspectorFactory() {
+		return integerInspectorFactory;
+	}
 
-    public IExpressionRuntimeProvider getExpressionRuntimeProvider() {
-        return expressionRuntimeProvider;
-    }
+	public IPrinterFactoryProvider getPrinterFactoryProvider() {
+		return printerFactoryProvider;
+	}
 
-    public IOperatorSchema getSchema(ILogicalOperator op) {
-        return schemaMap.get(op);
-    }
+	public IExpressionRuntimeProvider getExpressionRuntimeProvider() {
+		return expressionRuntimeProvider;
+	}
 
-    public void putSchema(ILogicalOperator op, IOperatorSchema schema) {
-        schemaMap.put(op, schema);
-    }
+	public IOperatorSchema getSchema(ILogicalOperator op) {
+		return schemaMap.get(op);
+	}
 
-    public LogicalVariable createNewVar() {
-        varCounter++;
-        LogicalVariable var = new LogicalVariable(-varCounter);
-        return var;
-    }
+	public void putSchema(ILogicalOperator op, IOperatorSchema schema) {
+		schemaMap.put(op, schema);
+	}
 
-    public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env) throws AlgebricksException {
-        return expressionTypeComputer.getType(expr, typingContext.getMetadataProvider(), env);
-    }
+	public LogicalVariable createNewVar() {
+		varCounter++;
+		LogicalVariable var = new LogicalVariable(-varCounter);
+		return var;
+	}
 
-    public INullWriterFactory getNullWriterFactory() {
-        return nullWriterFactory;
-    }
+	public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env)
+			throws AlgebricksException {
+		return expressionTypeComputer.getType(expr,
+				typingContext.getMetadataProvider(), env);
+	}
 
-    public INormalizedKeyComputerFactoryProvider getNormalizedKeyComputerFactoryProvider() {
-        return normalizedKeyComputerFactoryProvider;
-    }
+	public INullWriterFactory getNullWriterFactory() {
+		return nullWriterFactory;
+	}
 
-    public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
-        return expressionEvalSizeComputer;
-    }
+	public INormalizedKeyComputerFactoryProvider getNormalizedKeyComputerFactoryProvider() {
+		return normalizedKeyComputerFactoryProvider;
+	}
 
-    public int getFrameSize() {
-        return frameSize;
-    }
+	public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
+		return expressionEvalSizeComputer;
+	}
 
-    public IPartialAggregationTypeComputer getPartialAggregationTypeComputer() {
-        return partialAggregationTypeComputer;
-    }
+	public int getFrameSize() {
+		return frameSize;
+	}
 
-    public IVariableTypeEnvironment getTypeEnvironment(ILogicalOperator op) {
-        return typingContext.getOutputTypeEnvironment(op);
-    }
+	public IPartialAggregationTypeComputer getPartialAggregationTypeComputer() {
+		return partialAggregationTypeComputer;
+	}
+
+	public IVariableTypeEnvironment getTypeEnvironment(ILogicalOperator op) {
+		return typingContext.getOutputTypeEnvironment(op);
+	}
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index 1c250db..c18d554 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
@@ -31,6 +32,7 @@
 import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -94,7 +96,20 @@
         }
         return funFactories;
     }
-    
+
+    public static IBinaryHashFunctionFamily[] variablesToBinaryHashFunctionFamilies(
+            Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
+            throws AlgebricksException {
+        IBinaryHashFunctionFamily[] funFamilies = new IBinaryHashFunctionFamily[varLogical.size()];
+        int i = 0;
+        IBinaryHashFunctionFamilyProvider bhffProvider = context.getBinaryHashFunctionFamilyProvider();
+        for (LogicalVariable var : varLogical) {
+            Object type = env.getVarType(var);
+            funFamilies[i++] = bhffProvider.getBinaryHashFunctionFamily(type);
+        }
+        return funFamilies;
+    }
+
     public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(
             Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
             throws AlgebricksException {
@@ -107,15 +122,14 @@
         }
         return compFactories;
     }
-    
-    public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(
-            List<LogicalVariable> varLogical, int start, int size, IVariableTypeEnvironment env, JobGenContext context)
-            throws AlgebricksException {
+
+    public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(List<LogicalVariable> varLogical,
+            int start, int size, IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
         IBinaryComparatorFactory[] compFactories = new IBinaryComparatorFactory[size];
         IBinaryComparatorFactoryProvider bcfProvider = context.getBinaryComparatorFactoryProvider();
         for (int i = 0; i < size; i++) {
-                Object type = env.getVarType(varLogical.get(start + i));
-                compFactories[i] = bcfProvider.getBinaryComparatorFactory(type, true);
+            Object type = env.getVarType(varLogical.get(start + i));
+            compFactories[i] = bcfProvider.getBinaryComparatorFactory(type, true);
         }
         return compFactories;
     }
@@ -144,15 +158,14 @@
         }
         return typeTraits;
     }
-    
-    public static ITypeTraits[] variablesToTypeTraits(
-            List<LogicalVariable> varLogical, int start, int size, IVariableTypeEnvironment env, JobGenContext context)
-            throws AlgebricksException {
+
+    public static ITypeTraits[] variablesToTypeTraits(List<LogicalVariable> varLogical, int start, int size,
+            IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
         ITypeTraits[] typeTraits = new ITypeTraits[size];
         ITypeTraitProvider typeTraitProvider = context.getTypeTraitProvider();
         for (int i = 0; i < size; i++) {
-                Object type = env.getVarType(varLogical.get(start + i));
-                typeTraits[i] = typeTraitProvider.getTypeTrait(type);
+            Object type = env.getVarType(varLogical.get(start + i));
+            typeTraits[i] = typeTraitProvider.getTypeTrait(type);
         }
         return typeTraits;
     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index 7c63e01..738fc7f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -135,6 +135,7 @@
     /*
      * returns true if op1 and op2 have already been compared
      */
+    @Override
     public boolean checkAndAddToAlreadyCompared(ILogicalOperator op1, ILogicalOperator op2) {
         HashSet<ILogicalOperator> ops = alreadyCompared.get(op1);
         if (ops == null) {
@@ -151,6 +152,11 @@
             }
         }
     }
+    
+    @Override
+    public void removeFromAlreadyCompared(ILogicalOperator op1) {
+        alreadyCompared.remove(op1);
+    }
 
     public void addNotToBeInlinedVar(LogicalVariable var) {
         notToBeInlinedVars.add(var);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9ce910b..fc6c198 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -17,8 +17,8 @@
     public PhysicalOptimizationConfig() {
         int frameSize = 32768;
         setInt(FRAMESIZE, frameSize);
-        setInt(MAX_FRAMES_EXTERNAL_SORT, (int) (((long) 512 * MB) / frameSize));
-        setInt(MAX_FRAMES_EXTERNAL_GROUP_BY, (int) (((long) 256 * MB) / frameSize));
+        setInt(MAX_FRAMES_EXTERNAL_SORT, (int) (((long) 32 * MB) / frameSize));
+        setInt(MAX_FRAMES_EXTERNAL_GROUP_BY, (int) (((long) 32 * MB) / frameSize));
 
         // use http://www.rsok.com/~jrm/printprimes.html to find prime numbers
         setInt(DEFAULT_HASH_GROUP_TABLE_SIZE, 10485767);
diff --git a/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IBinaryHashFunctionFamilyProvider.java b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IBinaryHashFunctionFamilyProvider.java
new file mode 100644
index 0000000..8a992b3
--- /dev/null
+++ b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IBinaryHashFunctionFamilyProvider.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.data;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+
+public interface IBinaryHashFunctionFamilyProvider {
+
+	public IBinaryHashFunctionFamily getBinaryHashFunctionFamily(Object type)
+			throws AlgebricksException;
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
index 4f6699a..3daf85d 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
@@ -69,7 +69,7 @@
         VariableUtilities.getUsedVariables(op, varsUsedInUnnest);
 
         HashSet<LogicalVariable> producedInSubplan = new HashSet<LogicalVariable>();
-        VariableUtilities.getLiveVariables(subplan, producedInSubplan);
+        VariableUtilities.getProducedVariables(subplan, producedInSubplan);
 
         if (!producedInSubplan.containsAll(varsUsedInUnnest)) {
             return false;
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
index 4521d1a..fa5000e 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -149,9 +149,15 @@
             // Trivially joinable.
             return true;
         }
-        if (!belowSecondUnnest && op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-            // Bail on subplan.
-            return false;
+        if (!belowSecondUnnest) {
+            // Bail on the following operators.
+            switch (op.getOperatorTag()) {
+                case AGGREGATE:
+                case SUBPLAN:
+                case GROUP:
+                case UNNEST_MAP:
+                    return false;
+            }
         }
         switch (op.getOperatorTag()) {
             case UNNEST:
@@ -211,7 +217,8 @@
                 for (LogicalVariable producedVar : producedVars) {
                     if (outerUsedVars.contains(producedVar)) {
                         outerMatches++;
-                    } else if (innerUsedVars.contains(producedVar)) {
+                    }
+                    if (innerUsedVars.contains(producedVar)) {
                         innerMatches++;
                     }
                 }
@@ -221,24 +228,30 @@
                     // All produced vars used by outer partition.
                     outerOps.add(op);
                     targetUsedVars = outerUsedVars;
-                } else if (innerMatches == producedVars.size() && !producedVars.isEmpty()) {
+                }
+                if (innerMatches == producedVars.size() && !producedVars.isEmpty()) {
                     // All produced vars used by inner partition.
                     innerOps.add(op);
                     targetUsedVars = innerUsedVars;
-                } else if (innerMatches == 0 && outerMatches == 0) {
+                }
+                if (innerMatches == 0 && outerMatches == 0) {
                     // Op produces variables that are not used in the part of the plan we've seen (or it doesn't produce any vars).
                     // Try to figure out where it belongs by analyzing the used variables.
                     List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
                     VariableUtilities.getUsedVariables(op, usedVars);
                     for (LogicalVariable usedVar : usedVars) {
+                        boolean canBreak = false;
                         if (outerUsedVars.contains(usedVar)) {
                             outerOps.add(op);
                             targetUsedVars = outerUsedVars;
-                            break;
+                            canBreak = true;
                         }
                         if (innerUsedVars.contains(usedVar)) {
                             innerOps.add(op);
                             targetUsedVars = innerUsedVars;
+                            canBreak = true;
+                        }
+                        if (canBreak) {
                             break;
                         }
                     }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
new file mode 100644
index 0000000..f017e0f
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -0,0 +1,439 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+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.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Factors out common sub-expressions by assigning them to a variables, and replacing the common sub-expressions with references to those variables.
+ *
+ * Preconditions/Assumptions:
+ * Assumes no projects are in the plan. This rule ignores variable reference expressions and constants (other rules deal with those separately).
+ * 
+ * Postconditions/Examples:
+ * Plan with extracted sub-expressions. Generates one assign operator per extracted expression.
+ * 
+ * Example 1 - Simple Arithmetic Example (simplified)
+ * 
+ * Before plan:
+ * assign [$$1] <- [5 + 6 - 10]
+ *   assign [$$0] <- [5 + 6 + 30]
+ * 
+ * After plan:
+ * assign [$$1] <- [$$5 - 10]
+ *   assign [$$0] <- [$$5 + 30]
+ *     assign [$$5] <- [5 + 6]
+ * 
+ * Example 2 - Cleaning up 'Distinct By' (simplified)
+ * 
+ * Before plan: (notice how $$0 is not live after the distinct)
+ * assign [$$3] <- [field-access($$0, 1)]
+ *   distinct ([%0->$$5])
+ *     assign [$$5] <- [field-access($$0, 1)]
+ *       unnest $$0 <- [scan-dataset]
+ * 
+ * After plan: (notice how the issue of $$0 is fixed)
+ * assign [$$3] <- [$$5]
+ *   distinct ([$$5])
+ *     assign [$$5] <- [field-access($$0, 1)]
+ *       unnest $$0 <- [scan-dataset]
+ * 
+ * Example 3 - Pulling Common Expressions Above Joins (simplified)
+ * 
+ * Before plan:
+ * assign [$$9] <- funcZ(funcY($$8))
+ *   join (funcX(funcY($$8)))
+ * 
+ * After plan:
+ * assign [$$9] <- funcZ($$10))
+ *   select (funcX($$10))
+ *     assign [$$10] <- [funcY($$8)]
+ *       join (TRUE)
+ */
+public class ExtractCommonExpressionsRule implements IAlgebraicRewriteRule {
+
+    private final List<ILogicalExpression> originalAssignExprs = new ArrayList<ILogicalExpression>();
+    
+    private final CommonExpressionSubstitutionVisitor substVisitor = new CommonExpressionSubstitutionVisitor();
+    private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap = new HashMap<ILogicalExpression, ExprEquivalenceClass>();
+    
+    // Set of operators for which common subexpression elimination should not be performed.
+    private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<LogicalOperatorTag>();
+    static {
+        ignoreOps.add(LogicalOperatorTag.UNNEST);
+        ignoreOps.add(LogicalOperatorTag.UNNEST_MAP);
+        ignoreOps.add(LogicalOperatorTag.ORDER);
+        ignoreOps.add(LogicalOperatorTag.PROJECT);
+        ignoreOps.add(LogicalOperatorTag.AGGREGATE);
+        ignoreOps.add(LogicalOperatorTag.RUNNINGAGGREGATE);
+    }
+    
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        exprEqClassMap.clear();
+        substVisitor.setContext(context);
+        boolean modified = removeCommonExpressions(opRef, context);
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(opRef.getValue());
+        }
+        return modified;
+    }
+
+    private void updateEquivalenceClassMap(LogicalVariable lhs, Mutable<ILogicalExpression> rhsExprRef, ILogicalExpression rhsExpr, ILogicalOperator op) {
+        ExprEquivalenceClass exprEqClass = exprEqClassMap.get(rhsExpr);
+        if (exprEqClass == null) {
+            exprEqClass = new ExprEquivalenceClass(op, rhsExprRef);
+            exprEqClassMap.put(rhsExpr, exprEqClass);
+        }
+        exprEqClass.setVariable(lhs);
+    }
+
+    private boolean removeCommonExpressions(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+            return false;
+        }
+        
+        boolean modified = false;
+        // Recurse into children.
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            if (removeCommonExpressions(inputOpRef, context)) {
+                modified = true;
+            }
+        }
+        
+        // TODO: Deal with replicate properly. Currently, we just clear the expr equivalence map, since we want to avoid incorrect expression replacement
+        // (the resulting new variables should be assigned live below a replicate).
+        if (op.getOperatorTag() == LogicalOperatorTag.REPLICATE) {
+            exprEqClassMap.clear();
+            return modified;
+        }
+        // Exclude these operators.
+        if (ignoreOps.contains(op.getOperatorTag())) {
+            return modified;
+        }
+        
+        // Remember a copy of the original assign expressions, so we can add them to the equivalence class map
+        // after replacing expressions within the assign operator itself.
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            originalAssignExprs.clear();
+            int numVars = assignOp.getVariables().size();
+            for (int i = 0; i < numVars; i++) {
+                Mutable<ILogicalExpression> exprRef = assignOp.getExpressions().get(i);
+                ILogicalExpression expr = exprRef.getValue();
+                originalAssignExprs.add(expr.cloneExpression());
+            }
+        }
+        
+        // Perform common subexpression elimination.
+        substVisitor.setOperator(op);
+        if (op.acceptExpressionTransform(substVisitor)) {
+            modified = true;
+        }
+        
+        // Update equivalence class map.
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            int numVars = assignOp.getVariables().size();
+            for (int i = 0; i < numVars; i++) {
+                Mutable<ILogicalExpression> exprRef = assignOp.getExpressions().get(i);
+                ILogicalExpression expr = exprRef.getValue();
+                if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE
+                        || expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                    continue;
+                }
+                // Update equivalence class map.
+                LogicalVariable lhs = assignOp.getVariables().get(i);
+                updateEquivalenceClassMap(lhs, exprRef, exprRef.getValue(), op);
+                
+                // Update equivalence class map with original assign expression.
+                updateEquivalenceClassMap(lhs, exprRef, originalAssignExprs.get(i), op);
+            }
+        }
+
+        // TODO: For now do not perform replacement in nested plans
+        // due to the complication of figuring out whether the firstOp in an equivalence class is within a subplan, 
+        // and the resulting variable will not be visible to the outside.
+        // Since subplans should be eliminated in most cases, this behavior is acceptable for now.
+        /*
+        if (op.hasNestedPlans()) {
+            AbstractOperatorWithNestedPlans opWithNestedPlan = (AbstractOperatorWithNestedPlans) op;
+            for (ILogicalPlan nestedPlan : opWithNestedPlan.getNestedPlans()) {
+                for (Mutable<ILogicalOperator> rootRef : nestedPlan.getRoots()) {
+                    if (removeCommonExpressions(rootRef, context)) {
+                        modified = true;
+                    }
+                }
+            }
+        }
+        */
+
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            context.addToDontApplySet(this, op);
+        }
+        return modified;
+    }
+
+    private class CommonExpressionSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
+                
+        private final Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+        private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+        private IOptimizationContext context;
+        private ILogicalOperator op;        
+        
+        public void setContext(IOptimizationContext context) {
+            this.context = context;
+        }
+        
+        public void setOperator(ILogicalOperator op) throws AlgebricksException {
+            this.op = op;
+            liveVars.clear();
+            usedVars.clear();
+        }
+        
+        @Override
+        public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
+            if (liveVars.isEmpty() && usedVars.isEmpty()) {
+                VariableUtilities.getLiveVariables(op, liveVars);
+                VariableUtilities.getUsedVariables(op, usedVars);
+            }
+            
+            AbstractLogicalExpression expr = (AbstractLogicalExpression) exprRef.getValue();
+            boolean modified = false;
+            ExprEquivalenceClass exprEqClass = exprEqClassMap.get(expr);
+            if (exprEqClass != null) {
+                // Replace common subexpression with existing variable. 
+                if (exprEqClass.variableIsSet()) {
+                    // Check if the replacing variable is live at this op.
+                    // However, if the op is already using variables that are not live, then a replacement may enable fixing the plan.
+                    // This behavior is necessary to, e.g., properly deal with distinct by.
+                    // Also just replace the expr if we are replacing common exprs from within the same operator.
+                    if (liveVars.contains(exprEqClass.getVariable()) || !liveVars.containsAll(usedVars)
+                            || op == exprEqClass.getFirstOperator()) {
+                        exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                        // Do not descend into children since this expr has been completely replaced.
+                        return true;
+                    }
+                } else {
+                    if (assignCommonExpression(exprEqClass, expr)) {
+                        exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                        // Do not descend into children since this expr has been completely replaced.
+                        return true;
+                    }
+                }
+            } else {
+                if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE
+                        && expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                    exprEqClass = new ExprEquivalenceClass(op, exprRef);
+                    exprEqClassMap.put(expr, exprEqClass);
+                }
+            }
+            
+            // Descend into function arguments.
+            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+                for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                    if (transform(arg)) {
+                        modified = true;
+                    }
+                }
+            }
+            return modified;
+        }
+        
+        private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr) throws AlgebricksException {
+            AbstractLogicalOperator firstOp = (AbstractLogicalOperator) exprEqClass.getFirstOperator();
+            Mutable<ILogicalExpression> firstExprRef = exprEqClass.getFirstExpression();
+            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+                // Do not extract common expressions from within the same join operator.
+                if (firstOp == op) {
+                    return false;
+                }
+                AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) firstOp;
+                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();                
+                ILogicalExpression enclosingExpr = getEnclosingExpression(joinCond, firstExprRef.getValue());
+                if (enclosingExpr == null) {
+                    // No viable enclosing expression that we can pull out from the join.
+                    return false;
+                }
+                // Place a Select operator beneath op that contains the enclosing expression.
+                SelectOperator selectOp = new SelectOperator(new MutableObject<ILogicalExpression>(enclosingExpr));
+                selectOp.getInputs().add(new MutableObject<ILogicalOperator>(op.getInputs().get(0).getValue()));
+                op.getInputs().get(0).setValue(selectOp);
+                // Set firstOp to be the select below op, since we want to assign the common subexpr there.
+                firstOp = (AbstractLogicalOperator) selectOp;
+            } else if (firstOp.getInputs().size() > 1) { 
+                // Bail for any non-join operator with multiple inputs.
+                return false;
+            }                        
+            LogicalVariable newVar = context.newVar();
+            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(firstExprRef.getValue().cloneExpression()));            
+            // Place assign below firstOp.
+            newAssign.getInputs().add(new MutableObject<ILogicalOperator>(firstOp.getInputs().get(0).getValue()));
+            newAssign.setExecutionMode(firstOp.getExecutionMode());
+            firstOp.getInputs().get(0).setValue(newAssign);
+            // Replace original expr with variable reference, and set var in expression equivalence class.
+            firstExprRef.setValue(new VariableReferenceExpression(newVar));
+            exprEqClass.setVariable(newVar);
+            context.computeAndSetTypeEnvironmentForOperator(newAssign);
+            context.computeAndSetTypeEnvironmentForOperator(firstOp);
+            return true;
+        }
+
+        private ILogicalExpression getEnclosingExpression(Mutable<ILogicalExpression> conditionExprRef, ILogicalExpression commonSubExpr) {
+            ILogicalExpression conditionExpr = conditionExprRef.getValue();
+            if (conditionExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                return null;
+            }
+            if (isEqJoinCondition(commonSubExpr)) {
+                // Do not eliminate the common expression if we could use it for an equi-join.
+                return null;
+            }
+            AbstractFunctionCallExpression conditionFuncExpr = (AbstractFunctionCallExpression) conditionExpr;
+            // Boolean expression that encloses the common subexpression.
+            ILogicalExpression enclosingBoolExpr = null;
+            // We are not dealing with arbitrarily nested and/or expressions here.
+            FunctionIdentifier funcIdent = conditionFuncExpr.getFunctionIdentifier();
+            if (funcIdent.equals(AlgebricksBuiltinFunctions.AND) || funcIdent.equals(AlgebricksBuiltinFunctions.OR)) {
+                Iterator<Mutable<ILogicalExpression>> argIter = conditionFuncExpr.getArguments().iterator();
+                while (argIter.hasNext()) {
+                    Mutable<ILogicalExpression> argRef = argIter.next();
+                    if (containsExpr(argRef.getValue(), commonSubExpr)) {
+                        enclosingBoolExpr = argRef.getValue();
+                        // Remove the enclosing expression from the argument list.
+                        // We are going to pull it out into a new select operator.
+                        argIter.remove();
+                        break;
+                    }
+                }
+                // If and/or only has a single argument left, pull it out and remove the and/or function.
+                if (conditionFuncExpr.getArguments().size() == 1) {
+                    conditionExprRef.setValue(conditionFuncExpr.getArguments().get(0).getValue());
+                }
+            } else {
+                if (!containsExpr(conditionExprRef.getValue(), commonSubExpr)) {
+                    return null;
+                }
+                enclosingBoolExpr = conditionFuncExpr;
+                // Replace the enclosing expression with TRUE.
+                conditionExprRef.setValue(ConstantExpression.TRUE);
+            }
+            return enclosingBoolExpr;
+        }
+    }
+    
+    private boolean containsExpr(ILogicalExpression expr, ILogicalExpression searchExpr) {
+        if (expr == searchExpr) {
+            return true;
+        }
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        for (Mutable<ILogicalExpression> argRef : funcExpr.getArguments()) {
+            if (containsExpr(argRef.getValue(), searchExpr)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private boolean isEqJoinCondition(ILogicalExpression expr) {
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        if (funcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ)) {
+            ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+            ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+            if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+                    && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private final class ExprEquivalenceClass {
+        // First operator in which expression is used.
+        private final ILogicalOperator firstOp;
+        
+        // Reference to expression in first op.
+        private final Mutable<ILogicalExpression> firstExprRef;
+        
+        // Variable that this expression has been assigned to.
+        private LogicalVariable var;
+        
+        public ExprEquivalenceClass(ILogicalOperator firstOp, Mutable<ILogicalExpression> firstExprRef) {
+            this.firstOp = firstOp;
+            this.firstExprRef = firstExprRef;
+        }
+        
+        public ILogicalOperator getFirstOperator() {
+            return firstOp;
+        }
+        
+        public Mutable<ILogicalExpression> getFirstExpression() {
+            return firstExprRef;
+        }
+        
+        public void setVariable(LogicalVariable var) {
+            this.var = var;
+        }
+        
+        public LogicalVariable getVariable() {
+            return var;
+        }
+        
+        public boolean variableIsSet() {
+            return var != null;
+        }
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
new file mode 100644
index 0000000..df8ddda
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+/**
+ * Inlines variables that are referenced exactly once.
+ * 
+ * Preconditions/Assumptions:
+ * Assumes no projects are in the plan.
+ * 
+ * Example assuming variable $$3 is referenced exactly once.
+ * 
+ * Before plan:
+ * select (funcA($$3))
+ *   ...
+ *     assign [$$3] <- [field-access($$0, 1)]
+ * 
+ * After plan:
+ * select (funcA(field-access($$0, 1))
+ *   ...
+ *     assign [] <- []
+ */
+public class InlineSingleReferenceVariablesRule extends InlineVariablesRule {
+
+    // Maps from variable to a list of operators using that variable.
+    protected Map<LogicalVariable, List<ILogicalOperator>> usedVarsMap = new HashMap<LogicalVariable, List<ILogicalOperator>>();
+    protected List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+    
+    @Override
+    protected void prepare(IOptimizationContext context) {
+        super.prepare(context);
+        usedVarsMap.clear();
+        usedVars.clear();
+    }
+    
+    @Override
+    protected boolean performFinalAction() throws AlgebricksException {
+        boolean modified = false;
+        for (Map.Entry<LogicalVariable, List<ILogicalOperator>> entry : usedVarsMap.entrySet()) {
+            // Perform replacement only if variable is referenced a single time.
+            if (entry.getValue().size() == 1) {
+                ILogicalOperator op = entry.getValue().get(0);
+                if (!op.requiresVariableReferenceExpressions()) {
+                    inlineVisitor.setOperator(op);
+                    inlineVisitor.setTargetVariable(entry.getKey());
+                    if (op.acceptExpressionTransform(inlineVisitor)) {
+                        modified = true;
+                    }
+                    inlineVisitor.setTargetVariable(null);
+                }         
+            }
+        }
+        return modified;
+    }
+
+    @Override
+    protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+        usedVars.clear();
+        VariableUtilities.getUsedVariables(op, usedVars);
+        for (LogicalVariable var : usedVars) {
+            List<ILogicalOperator> opsUsingVar = usedVarsMap.get(var);
+            if (opsUsingVar == null) {
+                opsUsingVar = new ArrayList<ILogicalOperator>();
+                usedVarsMap.put(var, opsUsingVar);
+            }
+            opsUsingVar.add(op);
+        }
+        return false;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index 8a79d81..7fed577a 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -15,330 +15,231 @@
 package edu.uci.ics.hyracks.algebricks.rewriter.rules;
 
 import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 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.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
+/**
+ * Replaces variable reference expressions with their assigned function-call expression where applicable
+ * (some variables are generated by datasources).
+ * Inlining variables may enable other optimizations by allowing selects and assigns to be moved
+ * (e.g., a select may be pushed into a join to enable an efficient physical join operator).
+ * 
+ * Preconditions/Assumptions:
+ * Assumes no projects are in the plan. Only inlines variables whose assigned expression is a function call 
+ * (i.e., this rule ignores right-hand side constants and other variable references expressions  
+ * 
+ * Postconditions/Examples:
+ * All qualifying variables have been inlined.
+ * 
+ * Example (simplified):
+ * 
+ * Before plan:
+ * select <- [$$1 < $$2 + $$0]
+ *   assign [$$2] <- [funcZ() + $$0]
+ *     assign [$$0, $$1] <- [funcX(), funcY()]
+ * 
+ * After plan:
+ * select <- [funcY() < funcZ() + funcX() + funcX()]
+ *   assign [$$2] <- [funcZ() + funcX()]
+ *     assign [$$0, $$1] <- [funcX(), funcY()]
+ */
 public class InlineVariablesRule implements IAlgebraicRewriteRule {
 
+    // Map of variables that could be replaced by their producing expression.
+    // Populated during the top-down sweep of the plan.
+    protected Map<LogicalVariable, ILogicalExpression> varAssignRhs = new HashMap<LogicalVariable, ILogicalExpression>();
+
+    // Visitor for replacing variable reference expressions with their originating expression.
+    protected InlineVariablesVisitor inlineVisitor = new InlineVariablesVisitor(varAssignRhs);
+    
+    // Set of FunctionIdentifiers that we should not inline.
+    protected Set<FunctionIdentifier> doNotInlineFuncs = new HashSet<FunctionIdentifier>();
+    
+    protected boolean hasRun = false;
+    
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
     }
 
     @Override
-    /**
-     * 
-     * Does one big DFS sweep over the plan.
-     * 
-     */
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        if (hasRun) {
+            return false;
+        }
         if (context.checkIfInDontApplySet(this, opRef.getValue())) {
             return false;
         }
-        VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor(false);
-        VariableSubstitutionVisitor substVisitorForWrites = new VariableSubstitutionVisitor(true);
-        substVisitor.setContext(context);
-        substVisitorForWrites.setContext(context);
-        Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(opRef, context, true,
-                new LinkedList<EquivalenceClass>(), substVisitor, substVisitorForWrites);
-        return bb.first;
+        prepare(context);
+        boolean modified = inlineVariables(opRef, context);
+        if (performFinalAction()) {
+            modified = true;
+        }
+        hasRun = true;
+        return modified;
+    }
+    
+    protected void prepare(IOptimizationContext context) {
+        varAssignRhs.clear();
+        inlineVisitor.setContext(context);
+    }
+    
+    protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+        // Only inline variables in operators that can deal with arbitrary expressions.
+        if (!op.requiresVariableReferenceExpressions()) {
+            inlineVisitor.setOperator(op);
+            return op.acceptExpressionTransform(inlineVisitor);
+        }
+        return false;
     }
 
-    private Pair<Boolean, Boolean> collectEqClassesAndRemoveRedundantOps(Mutable<ILogicalOperator> opRef,
-            IOptimizationContext context, boolean first, List<EquivalenceClass> equivClasses,
-            VariableSubstitutionVisitor substVisitor, VariableSubstitutionVisitor substVisitorForWrites)
+    protected boolean performFinalAction() throws AlgebricksException {
+        return false;
+    }
+    
+    protected boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        // if (context.checkIfInDontApplySet(this, opRef.getValue())) {
-        // return false;
-        // }
-        if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
-            return new Pair<Boolean, Boolean>(false, false);
-        }
-        boolean modified = false;
-        boolean ecChange = false;
-        int cnt = 0;
-        for (Mutable<ILogicalOperator> i : op.getInputs()) {
-            boolean isOuterInputBranch = op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN && cnt == 1;
-            List<EquivalenceClass> eqc = isOuterInputBranch ? new LinkedList<EquivalenceClass>() : equivClasses;
-
-            Pair<Boolean, Boolean> bb = (collectEqClassesAndRemoveRedundantOps(i, context, false, eqc, substVisitor,
-                    substVisitorForWrites));
-
-            if (bb.first) {
-                modified = true;
-            }
-            if (bb.second) {
-                ecChange = true;
-            }
-
-            if (isOuterInputBranch) {
-                if (AlgebricksConfig.DEBUG) {
-                    AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- Equivalence classes for inner branch of outer op.: "
-                            + eqc + "\n");
-                }
-                for (EquivalenceClass ec : eqc) {
-                    if (!ec.representativeIsConst()) {
-                        equivClasses.add(ec);
-                    }
-                }
-            }
-
-            ++cnt;
-        }
-        if (op.hasNestedPlans()) {
-            AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op;
-            List<EquivalenceClass> eqc = equivClasses;
-            if (n.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-                eqc = new LinkedList<EquivalenceClass>();
-            } else {
-                eqc = equivClasses;
-            }
-            for (ILogicalPlan p : n.getNestedPlans()) {
-                for (Mutable<ILogicalOperator> r : p.getRoots()) {
-                    Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(r, context, false, eqc,
-                            substVisitor, substVisitorForWrites);
-                    if (bb.first) {
-                        modified = true;
-                    }
-                    if (bb.second) {
-                        ecChange = true;
-                    }
-                }
-            }
-        }
-        // we assume a variable is assigned a value only once
+        
+        // Update mapping from variables to expressions during top-down traversal.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator a = (AssignOperator) op;
-            ILogicalExpression rhs = a.getExpressions().get(0).getValue();
-            if (rhs.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                LogicalVariable varLeft = a.getVariables().get(0);
-                VariableReferenceExpression varRef = (VariableReferenceExpression) rhs;
-                LogicalVariable varRight = varRef.getVariableReference();
-
-                EquivalenceClass ecRight = findEquivClass(varRight, equivClasses);
-                if (ecRight != null) {
-                    ecRight.addMember(varLeft);
-                } else {
-                    List<LogicalVariable> m = new LinkedList<LogicalVariable>();
-                    m.add(varRight);
-                    m.add(varLeft);
-                    EquivalenceClass ec = new EquivalenceClass(m, varRight);
-                    equivClasses.add(ec);
-                    if (AlgebricksConfig.DEBUG) {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- New equivalence class: " + ec + "\n");
+            AssignOperator assignOp = (AssignOperator) op;
+            List<LogicalVariable> vars = assignOp.getVariables();
+            List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();            
+            for (int i = 0; i < vars.size(); i++) {
+                ILogicalExpression expr = exprs.get(i).getValue();
+                // Ignore functions that are in the doNotInline set.                
+                if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+                    if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier())) {
+                        continue;
                     }
                 }
-                ecChange = true;
-            } else if (((AbstractLogicalExpression) rhs).getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                LogicalVariable varLeft = a.getVariables().get(0);
-                List<LogicalVariable> m = new LinkedList<LogicalVariable>();
-                m.add(varLeft);
-                EquivalenceClass ec = new EquivalenceClass(m, (ConstantExpression) rhs);
-                // equivClassesForParent.add(ec);
-                equivClasses.add(ec);
-                ecChange = true;
-            }
-        } else if (op.getOperatorTag() == LogicalOperatorTag.GROUP && !(context.checkIfInDontApplySet(this, op))) {
-            GroupByOperator group = (GroupByOperator) op;
-            Pair<Boolean, Boolean> r1 = processVarExprPairs(group.getGroupByList(), equivClasses);
-            Pair<Boolean, Boolean> r2 = processVarExprPairs(group.getDecorList(), equivClasses);
-            modified = modified || r1.first || r2.first;
-            ecChange = r1.second || r2.second;
-        }
-        if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
-            assignVarsNeededByProject((ProjectOperator) op, equivClasses, context);
-        } else {
-            if (op.getOperatorTag() == LogicalOperatorTag.WRITE) {
-                substVisitorForWrites.setEquivalenceClasses(equivClasses);
-                if (op.acceptExpressionTransform(substVisitorForWrites)) {
-                    modified = true;
-                }
-            } else {
-                substVisitor.setEquivalenceClasses(equivClasses);
-                if (op.acceptExpressionTransform(substVisitor)) {
-                    modified = true;
-                    if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
-                        GroupByOperator group = (GroupByOperator) op;
-                        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gp : group.getGroupByList()) {
-                            if (gp.first != null
-                                    && gp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                LogicalVariable gv = ((VariableReferenceExpression) gp.second.getValue())
-                                        .getVariableReference();
-                                Iterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = group
-                                        .getDecorList().iterator();
-                                while (iter.hasNext()) {
-                                    Pair<LogicalVariable, Mutable<ILogicalExpression>> dp = iter.next();
-                                    if (dp.first == null
-                                            && dp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                        LogicalVariable dv = ((VariableReferenceExpression) dp.second.getValue())
-                                                .getVariableReference();
-                                        if (dv == gv) {
-                                            // The decor variable is redundant,
-                                            // since it is
-                                            // propagated as a grouping
-                                            // variable.
-                                            EquivalenceClass ec1 = findEquivClass(gv, equivClasses);
-                                            if (ec1 != null) {
-                                                ec1.addMember(gp.first);
-                                                ec1.setVariableRepresentative(gp.first);
-                                            } else {
-                                                List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
-                                                varList.add(gp.first);
-                                                varList.add(gv);
-                                                ec1 = new EquivalenceClass(varList, gp.first);
-                                            }
-                                            iter.remove();
-                                            break;
-                                        }
-                                    }
-                                }
-                            }
-                        }
-                    }
-                }
+                varAssignRhs.put(vars.get(i), exprs.get(i).getValue());
             }
         }
-        return new Pair<Boolean, Boolean>(modified, ecChange);
-    }
 
-    private Pair<Boolean, Boolean> processVarExprPairs(
-            List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> vePairs, List<EquivalenceClass> equivClasses) {
-        boolean ecFromGroup = false;
+        // Descend into children removing projects on the way.
         boolean modified = false;
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : vePairs) {
-            ILogicalExpression expr = p.second.getValue();
-            if (p.first != null && expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
-                LogicalVariable rhsVar = varRef.getVariableReference();
-                ecFromGroup = true;
-                EquivalenceClass ecRight = findEquivClass(rhsVar, equivClasses);
-                if (ecRight != null) {
-                    LogicalVariable replacingVar = ecRight.getVariableRepresentative();
-                    if (replacingVar != null && replacingVar != rhsVar) {
-                        varRef.setVariable(replacingVar);
-                        modified = true;
-                    }
-                }
-            }
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            if (inlineVariables(inputOpRef, context)) {
+                modified = true;
+            }            
         }
-        return new Pair<Boolean, Boolean>(modified, ecFromGroup);
+
+        if (performBottomUpAction(op)) {
+            modified = true;
+        }
+        
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            context.addToDontApplySet(this, op);
+            // Re-enable rules that we may have already tried. They could be applicable now after inlining.
+            context.removeFromAlreadyCompared(opRef.getValue());
+        }
+
+        return modified;
     }
 
-    // Instead of doing this, we could make Projection to be more expressive and
-    // also take constants (or even expression), at the expense of a more
-    // complex project push down.
-    private void assignVarsNeededByProject(ProjectOperator op, List<EquivalenceClass> equivClasses,
-            IOptimizationContext context) throws AlgebricksException {
-        List<LogicalVariable> prVars = op.getVariables();
-        int sz = prVars.size();
-        for (int i = 0; i < sz; i++) {
-            EquivalenceClass ec = findEquivClass(prVars.get(i), equivClasses);
-            if (ec != null) {
-                if (!ec.representativeIsConst()) {
-                    prVars.set(i, ec.getVariableRepresentative());
-                }
-            }
-        }
-    }
-
-    private final static EquivalenceClass findEquivClass(LogicalVariable var, List<EquivalenceClass> equivClasses) {
-        for (EquivalenceClass ec : equivClasses) {
-            if (ec.contains(var)) {
-                return ec;
-            }
-        }
-        return null;
-    }
-
-    private class VariableSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
-        private List<EquivalenceClass> equivClasses;
+    protected class InlineVariablesVisitor implements ILogicalExpressionReferenceTransform {
+        
+        private final Map<LogicalVariable, ILogicalExpression> varAssignRhs;
+        private final Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+        private final List<LogicalVariable> rhsUsedVars = new ArrayList<LogicalVariable>();        
+        private ILogicalOperator op;
         private IOptimizationContext context;
-        private final boolean doNotSubstWithConst;
-
-        public VariableSubstitutionVisitor(boolean doNotSubstWithConst) {
-            this.doNotSubstWithConst = doNotSubstWithConst;
+        // If set, only replace this variable reference.
+        private LogicalVariable targetVar;
+        
+        public InlineVariablesVisitor(Map<LogicalVariable, ILogicalExpression> varAssignRhs) {
+            this.varAssignRhs = varAssignRhs;
         }
-
+        
+        public void setTargetVariable(LogicalVariable targetVar) {
+            this.targetVar = targetVar;
+        }
+        
         public void setContext(IOptimizationContext context) {
             this.context = context;
         }
 
-        public void setEquivalenceClasses(List<EquivalenceClass> equivClasses) {
-            this.equivClasses = equivClasses;
+        public void setOperator(ILogicalOperator op) throws AlgebricksException {
+            this.op = op;
+            liveVars.clear();
         }
-
+        
         @Override
-        public boolean transform(Mutable<ILogicalExpression> exprRef) {
+        public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {            
             ILogicalExpression e = exprRef.getValue();
             switch (((AbstractLogicalExpression) e).getExpressionTag()) {
                 case VARIABLE: {
-                    // look for a required substitution
                     LogicalVariable var = ((VariableReferenceExpression) e).getVariableReference();
+                    // Restrict replacement to targetVar if it has been set.
+                    if (targetVar != null && var != targetVar) {
+                        return false;
+                    }
+                    // Make sure has not been excluded from inlining.
                     if (context.shouldNotBeInlined(var)) {
                         return false;
                     }
-                    EquivalenceClass ec = findEquivClass(var, equivClasses);
-                    if (ec == null) {
+                    ILogicalExpression rhs = varAssignRhs.get(var);
+                    if (rhs == null) {
+                        // Variable was not produced by an assign.
                         return false;
                     }
-                    if (ec.representativeIsConst()) {
-                        if (doNotSubstWithConst) {
-                            return false;
-                        }
-                        exprRef.setValue(ec.getConstRepresentative());
-                        return true;
-                    } else {
-                        LogicalVariable r = ec.getVariableRepresentative();
-                        if (!r.equals(var)) {
-                            exprRef.setValue(new VariableReferenceExpression(r));
-                            return true;
-                        } else {
+                    
+                    // Make sure used variables from rhs are live.
+                    if (liveVars.isEmpty()) {
+                        VariableUtilities.getLiveVariables(op, liveVars);
+                    }
+                    rhsUsedVars.clear();
+                    rhs.getUsedVariables(rhsUsedVars);
+                    for (LogicalVariable rhsUsedVar : rhsUsedVars) {
+                        if (!liveVars.contains(rhsUsedVar)) {
                             return false;
                         }
                     }
+                    
+                    // Replace variable reference with a clone of the rhs expr.
+                    exprRef.setValue(rhs.cloneExpression());
+                    return true;
                 }
                 case FUNCTION_CALL: {
                     AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) e;
-                    boolean m = false;
+                    boolean modified = false;
                     for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
                         if (transform(arg)) {
-                            m = true;
+                            modified = true;
                         }
                     }
-                    return m;
+                    return modified;
                 }
                 default: {
                     return false;
                 }
             }
         }
-
     }
 }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
index d54833e..431fca1 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
@@ -87,6 +87,7 @@
         projectOp.getInputs().add(new MutableObject<ILogicalOperator>(parentOp));
         opUnion.getInputs().get(branch).setValue(projectOp);
         projectOp.setPhysicalOperator(new StreamProjectPOperator());
+        context.computeAndSetTypeEnvironmentForOperator(projectOp);
         context.computeAndSetTypeEnvironmentForOperator(parentOp);
     }
 
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
new file mode 100644
index 0000000..a057f4f
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Projects away unused variables at the earliest possible point.
+ * Does a full DFS sweep of the plan adding ProjectOperators in the bottom-up pass.
+ * Also, removes projects that have become useless.
+ * TODO: This rule 'recklessly' adds as many projects as possible, but there is no guarantee
+ * that the overall cost of the plan is reduced since project operators also add a cost.
+ */
+public class IntroduceProjectsRule implements IAlgebraicRewriteRule {
+
+    private final Set<LogicalVariable> usedVars = new HashSet<LogicalVariable>();
+    private final Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+    private final Set<LogicalVariable> producedVars = new HashSet<LogicalVariable>();
+    private final List<LogicalVariable> projectVars = new ArrayList<LogicalVariable>();
+    protected boolean hasRun = false;
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        if (hasRun) {
+            return false;
+        }
+        hasRun = true;
+        return introduceProjects(null, -1, opRef, Collections.<LogicalVariable> emptySet(), context);
+    }
+
+    protected boolean introduceProjects(AbstractLogicalOperator parentOp, int parentInputIndex,
+            Mutable<ILogicalOperator> opRef, Set<LogicalVariable> parentUsedVars, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        boolean modified = false;
+
+        usedVars.clear();
+        VariableUtilities.getUsedVariables(op, usedVars);
+
+        // In the top-down pass, maintain a set of variables that are used in op and all its parents.
+        HashSet<LogicalVariable> parentsUsedVars = new HashSet<LogicalVariable>();
+        parentsUsedVars.addAll(parentUsedVars);
+        parentsUsedVars.addAll(usedVars);
+
+        // Descend into children.        
+        for (int i = 0; i < op.getInputs().size(); i++) {
+            Mutable<ILogicalOperator> inputOpRef = op.getInputs().get(i);
+            if (introduceProjects(op, i, inputOpRef, parentsUsedVars, context)) {
+                modified = true;
+            }
+        }
+
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+        }
+        // In the bottom-up pass, determine which live variables are not used by op's parents.
+        // Such variables are be projected away.
+        liveVars.clear();
+        VariableUtilities.getLiveVariables(op, liveVars);
+        producedVars.clear();
+        VariableUtilities.getProducedVariables(op, producedVars);
+        liveVars.removeAll(producedVars);
+
+        projectVars.clear();
+        for (LogicalVariable liveVar : liveVars) {
+            if (parentsUsedVars.contains(liveVar)) {
+                projectVars.add(liveVar);
+            }
+        }
+
+        // Some of the variables that are live at this op are not used above.
+        if (projectVars.size() != liveVars.size()) {
+            // Add a project operator under each of op's qualifying input branches.
+            for (int i = 0; i < op.getInputs().size(); i++) {
+                ILogicalOperator childOp = op.getInputs().get(i).getValue();
+                liveVars.clear();
+                VariableUtilities.getLiveVariables(childOp, liveVars);
+                List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+                vars.addAll(projectVars);
+                // Only retain those variables that are live in the i-th input branch.
+                vars.retainAll(liveVars);
+                if (vars.size() != liveVars.size()) {
+                    ProjectOperator projectOp = new ProjectOperator(vars);
+                    projectOp.getInputs().add(new MutableObject<ILogicalOperator>(childOp));
+                    op.getInputs().get(i).setValue(projectOp);
+                    context.computeAndSetTypeEnvironmentForOperator(projectOp);
+                    modified = true;
+                }
+            }
+        } else if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+            // Check if the existing project has become useless.
+            liveVars.clear();
+            VariableUtilities.getLiveVariables(op.getInputs().get(0).getValue(), liveVars);
+            ProjectOperator projectOp = (ProjectOperator) op;
+            List<LogicalVariable> projectVars = projectOp.getVariables();
+            if (liveVars.size() == projectVars.size() && liveVars.containsAll(projectVars)) {
+                boolean eliminateProject = true;
+                // For UnionAll the variables must also be in exactly the correct order.
+                if (parentOp.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
+                    eliminateProject = canEliminateProjectBelowUnion((UnionAllOperator) parentOp, projectOp,
+                            parentInputIndex);
+                }
+                if (eliminateProject) {
+                    // The existing project has become useless. Remove it.
+                    parentOp.getInputs().get(parentInputIndex).setValue(op.getInputs().get(0).getValue());
+                }
+            }
+        }
+
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+        }
+        return modified;
+    }
+    
+    private boolean canEliminateProjectBelowUnion(UnionAllOperator unionOp, ProjectOperator projectOp,
+            int unionInputIndex) throws AlgebricksException {
+        List<LogicalVariable> orderedLiveVars = new ArrayList<LogicalVariable>();
+        VariableUtilities.getLiveVariables(projectOp.getInputs().get(0).getValue(), orderedLiveVars);
+        int numVars = orderedLiveVars.size();
+        for (int i = 0; i < numVars; i++) {
+            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple = unionOp.getVariableMappings().get(i);
+            if (unionInputIndex == 0) {
+                if (varTriple.first != orderedLiveVars.get(i)) {
+                    return false;
+                }
+            } else {
+                if (varTriple.second != orderedLiveVars.get(i)) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index 75862cf..8b4f0a1 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -38,9 +38,6 @@
 
 public class PullSelectOutOfEqJoin implements IAlgebraicRewriteRule {
 
-    private List<Mutable<ILogicalExpression>> eqVarVarComps = new ArrayList<Mutable<ILogicalExpression>>();
-    private List<Mutable<ILogicalExpression>> otherPredicates = new ArrayList<Mutable<ILogicalExpression>>();
-
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         return false;
@@ -66,8 +63,8 @@
         if (!fi.equals(AlgebricksBuiltinFunctions.AND)) {
             return false;
         }
-        eqVarVarComps.clear();
-        otherPredicates.clear();
+        List<Mutable<ILogicalExpression>> eqVarVarComps = new ArrayList<Mutable<ILogicalExpression>>();
+        List<Mutable<ILogicalExpression>> otherPredicates = new ArrayList<Mutable<ILogicalExpression>>();
         for (Mutable<ILogicalExpression> arg : fexp.getArguments()) {
             if (isEqVarVar(arg.getValue())) {
                 eqVarVarComps.add(arg);
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
new file mode 100644
index 0000000..1bcf95a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
@@ -0,0 +1,149 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pushes an AssignOperator below a UnionAll operator by creating an new AssignOperator below each of 
+ * the UnionAllOperator's branches with appropriate variable replacements.
+ * This rule can help to enable other rules that are difficult to fire across a UnionAllOperator, 
+ * for example, eliminating common sub-expressions.
+ * 
+ * Example:
+ * 
+ * Before plan:
+ * ...
+ * assign [$$20, $$21] <- [funcA($$3), funcB($$6)]
+ *   union ($$1, $$2, $$3) ($$4, $$5, $$6)
+ *     union_branch_0
+ *       ...
+ *     union_branch_1
+ *       ...
+ *     
+ * After plan:
+ * ...
+ * union ($$1, $$2, $$3) ($$4, $$5, $$6) ($$22, $$24, $$20) ($$23, $$25, $$21)
+ *   assign [$$22, $$23] <- [funcA($$1), funcB($$4)]
+ *     union_branch_0
+ *       ...
+ *   assign [$$24, $$25] <- [funcA($$2), funcB($$5)]
+ *     union_branch_1
+ *       ...
+ */
+public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (!op.hasInputs()) {
+            return false;
+        }
+
+        boolean modified = false;
+        for (int i = 0; i < op.getInputs().size(); i++) {
+            AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+            if (childOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+                continue;
+            }
+            AssignOperator assignOp = (AssignOperator) childOp;
+
+            AbstractLogicalOperator childOfChildOp = (AbstractLogicalOperator) assignOp.getInputs().get(0).getValue();
+            if (childOfChildOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
+                continue;
+            }
+            UnionAllOperator unionOp = (UnionAllOperator) childOfChildOp;
+
+            Set<LogicalVariable> assignUsedVars = new HashSet<LogicalVariable>();
+            VariableUtilities.getUsedVariables(assignOp, assignUsedVars);
+
+            List<LogicalVariable> assignVars = assignOp.getVariables();
+
+            AssignOperator[] newAssignOps = new AssignOperator[2];
+            for (int j = 0; j < unionOp.getInputs().size(); j++) {
+                newAssignOps[j] = createAssignBelowUnionAllBranch(unionOp, j, assignOp, assignUsedVars, context);
+            }
+            // Add original assign variables to the union variable mappings.
+            for (int j = 0; j < assignVars.size(); j++) {
+                LogicalVariable first = newAssignOps[0].getVariables().get(j);
+                LogicalVariable second = newAssignOps[1].getVariables().get(j);
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+                        first, second, assignVars.get(j));
+                unionOp.getVariableMappings().add(varMapping);
+            }
+            context.computeAndSetTypeEnvironmentForOperator(unionOp);
+
+            // Remove original assign operator.
+            op.getInputs().set(i, assignOp.getInputs().get(0));
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            modified = true;
+        }
+
+        return modified;
+    }
+
+    private AssignOperator createAssignBelowUnionAllBranch(UnionAllOperator unionOp, int inputIndex,
+            AssignOperator originalAssignOp, Set<LogicalVariable> assignUsedVars, IOptimizationContext context)
+            throws AlgebricksException {
+        AssignOperator newAssignOp = cloneAssignOperator(originalAssignOp, context);
+        newAssignOp.getInputs()
+                .add(new MutableObject<ILogicalOperator>(unionOp.getInputs().get(inputIndex).getValue()));
+        context.computeAndSetTypeEnvironmentForOperator(newAssignOp);
+        unionOp.getInputs().get(inputIndex).setValue(newAssignOp);
+        int numVarMappings = unionOp.getVariableMappings().size();
+        for (int i = 0; i < numVarMappings; i++) {
+            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping = unionOp.getVariableMappings().get(i);
+            if (assignUsedVars.contains(varMapping.third)) {
+                LogicalVariable replacementVar;
+                if (inputIndex == 0) {
+                    replacementVar = varMapping.first;
+                } else {
+                    replacementVar = varMapping.second;
+                }
+                VariableUtilities.substituteVariables(newAssignOp, varMapping.third, replacementVar, context);
+            }
+        }
+        return newAssignOp;
+    }
+
+    /**
+     * Clones the given assign operator changing the returned variables to be new ones.
+     * Also, leaves the inputs of the clone clear.
+     */
+    private AssignOperator cloneAssignOperator(AssignOperator assignOp, IOptimizationContext context) {
+        List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+        List<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+        int numVars = assignOp.getVariables().size();
+        for (int i = 0; i < numVars; i++) {
+            vars.add(context.newVar());
+            exprs.add(new MutableObject<ILogicalExpression>(assignOp.getExpressions().get(i).getValue()
+                    .cloneExpression()));
+        }
+        AssignOperator assignCloneOp = new AssignOperator(vars, exprs);
+        assignCloneOp.setExecutionMode(assignOp.getExecutionMode());
+        return assignCloneOp;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
new file mode 100644
index 0000000..16b010e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+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.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pushes function-call expressions below a join if possible.
+ * Assigns the result of such function-calls expressions to new variables, and replaces the original
+ * expression with a corresponding variable reference expression.
+ * This rule can help reduce the cost of computing expensive functions by pushing them below
+ * a join (which may blow up the cardinality).
+ * Also, this rule may help to enable other rules such as common subexpression elimination, again to reduce
+ * the number of calls to expensive functions.
+ * 
+ * Example: (we are pushing pushMeFunc)
+ * 
+ * Before plan:
+ * assign [$$10] <- [funcA(funcB(pushMeFunc($$3, $$4)))]
+ *   join (some condition) 
+ *     join_branch_0 where $$3 and $$4 are not live
+ *       ...
+ *     join_branch_1 where $$3 and $$4 are live
+ *       ...
+ * 
+ * After plan:
+ * assign [$$10] <- [funcA(funcB($$11))]
+ *   join (some condition) 
+ *     join_branch_0 where $$3 and $$4 are not live
+ *       ...
+ *     join_branch_1 where $$3 and $$4 are live
+ *       assign[$$11] <- [pushMeFunc($$3, $$4)]
+ *         ...
+ */
+public class PushFunctionsBelowJoin implements IAlgebraicRewriteRule {
+
+    private final Set<FunctionIdentifier> toPushFuncIdents;
+    private final List<Mutable<ILogicalExpression>> funcExprs = new ArrayList<Mutable<ILogicalExpression>>();
+    private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+    private final List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+
+    public PushFunctionsBelowJoin(Set<FunctionIdentifier> toPushFuncIdents) {
+        this.toPushFuncIdents = toPushFuncIdents;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+        AssignOperator assignOp = (AssignOperator) op;
+
+        // Find a join operator below this assign.
+        Mutable<ILogicalOperator> joinOpRef = findJoinOp(assignOp.getInputs().get(0));
+        if (joinOpRef == null) {
+            return false;
+        }
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinOpRef.getValue();
+
+        // Check if the assign uses a function that we wish to push below the join if possible.
+        funcExprs.clear();
+        gatherFunctionCalls(assignOp, funcExprs);
+        if (funcExprs.isEmpty()) {
+            return false;
+        }
+
+        // Try to push the functions down the input branches of the join.
+        boolean modified = false;
+        if (pushDownFunctions(joinOp, 0, funcExprs, context)) {
+            modified = true;
+        }
+        if (pushDownFunctions(joinOp, 1, funcExprs, context)) {
+            modified = true;
+        }
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(joinOp);
+        }
+        return modified;
+    }
+
+    private Mutable<ILogicalOperator> findJoinOp(Mutable<ILogicalOperator> opRef) {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        switch (op.getOperatorTag()) {
+            case INNERJOIN:
+            case LEFTOUTERJOIN: {
+                return opRef;
+            }
+            // Bail on these operators.
+            case GROUP:
+            case AGGREGATE:
+            case DISTINCT:
+            case UNNEST_MAP: {
+                return null;
+            }
+            // Traverse children.
+            default: {
+                for (Mutable<ILogicalOperator> childOpRef : op.getInputs()) {
+                    return findJoinOp(childOpRef);
+                }
+            }
+        }
+        return null;
+    }
+
+    private void gatherFunctionCalls(AssignOperator assignOp, List<Mutable<ILogicalExpression>> funcExprs) {
+        for (Mutable<ILogicalExpression> exprRef : assignOp.getExpressions()) {
+            gatherFunctionCalls(exprRef, funcExprs);
+        }
+    }
+
+    private void gatherFunctionCalls(Mutable<ILogicalExpression> exprRef, List<Mutable<ILogicalExpression>> funcExprs) {
+        AbstractLogicalExpression expr = (AbstractLogicalExpression) exprRef.getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return;
+        }
+        // Check whether the function is a function we want to push.
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        if (toPushFuncIdents.contains(funcExpr.getFunctionIdentifier())) {
+            funcExprs.add(exprRef);
+        }
+        // Traverse arguments.
+        for (Mutable<ILogicalExpression> funcArg : funcExpr.getArguments()) {
+            gatherFunctionCalls(funcArg, funcExprs);
+        }
+    }
+
+    private boolean pushDownFunctions(AbstractBinaryJoinOperator joinOp, int inputIndex,
+            List<Mutable<ILogicalExpression>> funcExprs, IOptimizationContext context) throws AlgebricksException {
+        ILogicalOperator joinInputOp = joinOp.getInputs().get(inputIndex).getValue();
+        liveVars.clear();
+        VariableUtilities.getLiveVariables(joinInputOp, liveVars);
+        Iterator<Mutable<ILogicalExpression>> funcIter = funcExprs.iterator();
+        List<LogicalVariable> assignVars = null;
+        List<Mutable<ILogicalExpression>> assignExprs = null;
+        while (funcIter.hasNext()) {
+            Mutable<ILogicalExpression> funcExprRef = funcIter.next();
+            ILogicalExpression funcExpr = funcExprRef.getValue();
+            usedVars.clear();
+            funcExpr.getUsedVariables(usedVars);
+            // Check if we can push the function down this branch.
+            if (liveVars.containsAll(usedVars)) {
+                if (assignVars == null) {
+                    assignVars = new ArrayList<LogicalVariable>();
+                    assignExprs = new ArrayList<Mutable<ILogicalExpression>>();
+                }
+                // Replace the original expression with a variable reference expression.
+                LogicalVariable replacementVar = context.newVar();
+                assignVars.add(replacementVar);
+                assignExprs.add(new MutableObject<ILogicalExpression>(funcExpr));
+                funcExprRef.setValue(new VariableReferenceExpression(replacementVar));
+                funcIter.remove();
+            }
+        }
+        // Create new assign operator below the join if any functions can be pushed.
+        if (assignVars != null) {
+            AssignOperator newAssign = new AssignOperator(assignVars, assignExprs);
+            newAssign.getInputs().add(new MutableObject<ILogicalOperator>(joinInputOp));
+            newAssign.setExecutionMode(joinOp.getExecutionMode());
+            joinOp.getInputs().get(inputIndex).setValue(newAssign);
+            context.computeAndSetTypeEnvironmentForOperator(newAssign);
+            return true;
+        }
+        return false;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index 8c679c5..99a6b8c 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -143,11 +143,11 @@
         if (!intersectsBranch[0] && !intersectsBranch[1]) {
             return false;
         }
+        if (needToPushOps) {
+            pushOps(pushedOnLeft, joinBranchLeftRef, context);
+            pushOps(pushedOnRight, joinBranchRightRef, context);
+        }
         if (intersectsAllBranches) {
-            if (needToPushOps) {
-                pushOps(pushedOnLeft, joinBranchLeftRef, context);
-                pushOps(pushedOnRight, joinBranchRightRef, context);
-            }
             addCondToJoin(select, join, context);
         } else { // push down
             Iterator<Mutable<ILogicalOperator>> branchIter = join.getInputs().iterator();
@@ -156,13 +156,6 @@
                 Mutable<ILogicalOperator> branch = branchIter.next();
                 boolean inter = intersectsBranch[j];
                 if (inter) {
-                    if (needToPushOps) {
-                        if (j == 0) {
-                            pushOps(pushedOnLeft, joinBranchLeftRef, context);
-                        } else {
-                            pushOps(pushedOnRight, joinBranchRightRef, context);
-                        }
-                    }
                     copySelectToBranch(select, branch, context);
                 }
 
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantGroupByDecorVars.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantGroupByDecorVars.java
new file mode 100644
index 0000000..1106898
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantGroupByDecorVars.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Removes duplicate variables from a group-by operator's decor list.
+ */
+public class RemoveRedundantGroupByDecorVars implements IAlgebraicRewriteRule {
+
+    private final Set<LogicalVariable> vars = new HashSet<LogicalVariable>();
+    
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+            return false;
+        }
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        vars.clear();
+        
+        boolean modified = false;
+        GroupByOperator groupOp = (GroupByOperator) op;
+        Iterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = groupOp.getDecorList().iterator();
+        while (iter.hasNext()) {
+            Pair<LogicalVariable, Mutable<ILogicalExpression>> decor = iter.next();
+            if (decor.first != null || decor.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                continue;
+            }
+            VariableReferenceExpression varRefExpr = (VariableReferenceExpression) decor.second.getValue();
+            LogicalVariable var = varRefExpr.getVariableReference();
+            if (vars.contains(var)) {
+                iter.remove();
+                modified = true;
+            } else {
+                vars.add(var);
+            }
+        }
+        if (modified) {
+            context.addToDontApplySet(this, op);
+        }
+        return modified;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
new file mode 100644
index 0000000..ec57be5
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -0,0 +1,291 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Replaces redundant variable references with their bottom-most equivalent representative.
+ * Does a DFS sweep over the plan keeping track of variable equivalence classes.
+ * For example, this rule would perform the following rewrite.
+ * 
+ * Before Plan:
+ * select (function-call: func, Args:[%0->$$11])
+ *   project [$11]
+ *     assign [$$11] <- [$$10]
+ *       assign [$$10] <- [$$9]
+ *         assign [$$9] <- ...
+ *           ...
+ *           
+ * After Plan:
+ * select (function-call: func, Args:[%0->$$9])
+ *   project [$9]
+ *     assign [$$11] <- [$$9]
+ *       assign [$$10] <- [$$9]
+ *         assign [$$9] <- ...
+ *           ...
+ */
+public class RemoveRedundantVariablesRule implements IAlgebraicRewriteRule {
+
+    private final VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor();
+    private final Map<LogicalVariable, List<LogicalVariable>> equivalentVarsMap = new HashMap<LogicalVariable, List<LogicalVariable>>();
+
+    protected boolean hasRun = false;
+    
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+            return false;
+        }
+        equivalentVarsMap.clear();
+        boolean modified = removeRedundantVariables(opRef, context);
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(opRef.getValue());
+        }
+        return modified;
+    }
+
+    private void updateEquivalenceClassMap(LogicalVariable lhs, LogicalVariable rhs) {
+        List<LogicalVariable> equivalentVars = equivalentVarsMap.get(rhs);
+        if (equivalentVars == null) {
+            equivalentVars = new ArrayList<LogicalVariable>();
+            // The first element in the list is the bottom-most representative which will replace all equivalent vars.
+            equivalentVars.add(rhs);
+            equivalentVars.add(lhs);
+            equivalentVarsMap.put(rhs, equivalentVars);
+        }
+        equivalentVarsMap.put(lhs, equivalentVars);
+        equivalentVars.get(0);
+    }
+
+    private boolean removeRedundantVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        boolean modified = false;
+
+        // Recurse into children.
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            if (removeRedundantVariables(inputOpRef, context)) {
+                modified = true;
+            }
+        }
+
+        // Update equivalence class map.
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            int numVars = assignOp.getVariables().size();
+            for (int i = 0; i < numVars; i++) {
+                ILogicalExpression expr = assignOp.getExpressions().get(i).getValue();
+                if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                    continue;
+                }
+                VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
+                // Update equivalence class map.
+                LogicalVariable lhs = assignOp.getVariables().get(i);
+                LogicalVariable rhs = rhsVarRefExpr.getVariableReference();
+                updateEquivalenceClassMap(lhs, rhs);
+            }
+        }
+
+        // Replace variable references with their first representative. 
+        if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+            // The project operator does not use expressions, so we need to replace it's variables manually.
+            if (replaceProjectVars((ProjectOperator) op)) {
+                modified = true;
+            }
+        } else if(op.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
+            // Replace redundant variables manually in the UnionAll operator.
+            if (replaceUnionAllVars((UnionAllOperator) op)) {
+                modified = true;
+            }
+        } else {
+            if (op.acceptExpressionTransform(substVisitor)) {
+                modified = true;
+            }
+        }
+
+        // Perform variable replacement in nested plans. 
+        if (op.hasNestedPlans()) {
+            AbstractOperatorWithNestedPlans opWithNestedPlan = (AbstractOperatorWithNestedPlans) op;
+            for (ILogicalPlan nestedPlan : opWithNestedPlan.getNestedPlans()) {
+                for (Mutable<ILogicalOperator> rootRef : nestedPlan.getRoots()) {
+                    if (removeRedundantVariables(rootRef, context)) {
+                        modified = true;
+                    }
+                }
+            }
+        }
+
+        // Deal with re-mapping of variables in group by.
+        if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
+            if (handleGroupByVarRemapping((GroupByOperator) op)) {
+                modified = true;
+            }
+        }
+
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            context.addToDontApplySet(this, op);
+        }
+        return modified;
+    }
+
+    private boolean handleGroupByVarRemapping(GroupByOperator groupOp) {
+        boolean modified = false;
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gp : groupOp.getGroupByList()) {
+            if (gp.first == null || gp.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                continue;
+            }
+            LogicalVariable groupByVar = ((VariableReferenceExpression) gp.second.getValue()).getVariableReference();
+            Iterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = groupOp.getDecorList().iterator();
+            while (iter.hasNext()) {
+                Pair<LogicalVariable, Mutable<ILogicalExpression>> dp = iter.next();
+                if (dp.first != null || dp.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                    continue;
+                }
+                LogicalVariable dv = ((VariableReferenceExpression) dp.second.getValue()).getVariableReference();
+                if (dv == groupByVar) {
+                    // The decor variable is redundant, since it is propagated as a grouping variable.
+                    List<LogicalVariable> equivalentVars = equivalentVarsMap.get(groupByVar);
+                    if (equivalentVars != null) {
+                        // Change representative of this equivalence class.
+                        equivalentVars.set(0, gp.first);
+                        equivalentVarsMap.put(gp.first, equivalentVars);
+                    } else {
+                        updateEquivalenceClassMap(gp.first, groupByVar);
+                    }
+                    iter.remove();
+                    modified = true;
+                    break;
+                }
+            }
+        }
+        return modified;
+    }
+
+    /**
+     * Replace the projects's variables with their corresponding representative
+     * from the equivalence class map (if any).
+     * We cannot use the VariableSubstitutionVisitor here because the project ops
+     * maintain their variables as a list and not as expressions.
+     */
+    private boolean replaceProjectVars(ProjectOperator op) throws AlgebricksException {
+        List<LogicalVariable> vars = op.getVariables();
+        int size = vars.size();
+        boolean modified = false;
+        for (int i = 0; i < size; i++) {
+            LogicalVariable var = vars.get(i);
+            List<LogicalVariable> equivalentVars = equivalentVarsMap.get(var);
+            if (equivalentVars == null) {
+                continue;
+            }
+            // Replace with equivalence class representative.
+            LogicalVariable representative = equivalentVars.get(0);
+            if (representative != var) {
+                vars.set(i, equivalentVars.get(0));
+                modified = true;
+            }
+        }
+        return modified;
+    }
+
+    private boolean replaceUnionAllVars(UnionAllOperator op) throws AlgebricksException {
+        boolean modified = false;
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping : op.getVariableMappings()) {
+            List<LogicalVariable> firstEquivalentVars = equivalentVarsMap.get(varMapping.first);
+            List<LogicalVariable> secondEquivalentVars = equivalentVarsMap.get(varMapping.second);
+            // Replace variables with their representative.
+            if (firstEquivalentVars != null) {
+                varMapping.first = firstEquivalentVars.get(0);
+                modified = true;
+            }
+            if (secondEquivalentVars != null) {
+                varMapping.second = secondEquivalentVars.get(0);
+                modified = true;
+            }
+        }
+        return modified;
+    }
+    
+    private class VariableSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
+        @Override
+        public boolean transform(Mutable<ILogicalExpression> exprRef) {
+            ILogicalExpression e = exprRef.getValue();
+            switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+                case VARIABLE: {
+                    // Replace variable references with their equivalent representative in the equivalence class map.
+                    VariableReferenceExpression varRefExpr = (VariableReferenceExpression) e;
+                    LogicalVariable var = varRefExpr.getVariableReference();
+                    List<LogicalVariable> equivalentVars = equivalentVarsMap.get(var);
+                    if (equivalentVars == null) {
+                        return false;
+                    }
+                    LogicalVariable representative = equivalentVars.get(0);
+                    if (representative != var) {
+                        varRefExpr.setVariable(representative);
+                        return true;
+                    }
+                    return false;
+                }
+                case FUNCTION_CALL: {
+                    AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) e;
+                    boolean modified = false;
+                    for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                        if (transform(arg)) {
+                            modified = true;
+                        }
+                    }
+                    return modified;
+                }
+                default: {
+                    return false;
+                }
+            }
+        }
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
index c53ea0a..e0c2741 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -23,6 +23,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
@@ -33,10 +34,14 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
+/**
+ * Removes unused variables from Assign, Unnest, Aggregate, and UnionAll operators.
+ */
 public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule {
 
     @Override
@@ -55,7 +60,7 @@
         if (smthToRemove) {
             removeUnusedAssigns(opRef, toRemove, context);
         }
-        return smthToRemove;
+        return !toRemove.isEmpty();
     }
 
     private void removeUnusedAssigns(Mutable<ILogicalOperator> opRef, Set<LogicalVariable> toRemove,
@@ -87,28 +92,59 @@
 
     private int removeFromAssigns(AbstractLogicalOperator op, Set<LogicalVariable> toRemove,
             IOptimizationContext context) throws AlgebricksException {
-        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator assign = (AssignOperator) op;
-            if (removeUnusedVarsAndExprs(toRemove, assign.getVariables(), assign.getExpressions())) {
-                context.computeAndSetTypeEnvironmentForOperator(assign);
+        switch (op.getOperatorTag()) {
+            case ASSIGN: {
+                AssignOperator assign = (AssignOperator) op;
+                if (removeUnusedVarsAndExprs(toRemove, assign.getVariables(), assign.getExpressions())) {
+                    context.computeAndSetTypeEnvironmentForOperator(assign);
+                }
+                return assign.getVariables().size();
             }
-            return assign.getVariables().size();
-        } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
-            AggregateOperator agg = (AggregateOperator) op;
-            if (removeUnusedVarsAndExprs(toRemove, agg.getVariables(), agg.getExpressions())) {
-                context.computeAndSetTypeEnvironmentForOperator(agg);
+            case AGGREGATE: {
+                AggregateOperator agg = (AggregateOperator) op;
+                if (removeUnusedVarsAndExprs(toRemove, agg.getVariables(), agg.getExpressions())) {
+                    context.computeAndSetTypeEnvironmentForOperator(agg);
+                }
+                return agg.getVariables().size();
             }
-            return agg.getVariables().size();
-        } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
-            UnnestOperator uOp = (UnnestOperator) op;
-            LogicalVariable pVar = uOp.getPositionalVariable();
-            if (pVar != null && toRemove.contains(pVar)) {
-                uOp.setPositionalVariable(null);
+            case UNNEST: {
+                UnnestOperator uOp = (UnnestOperator) op;
+                LogicalVariable pVar = uOp.getPositionalVariable();
+                if (pVar != null && toRemove.contains(pVar)) {
+                    uOp.setPositionalVariable(null);
+                }
+                break;
+            }
+            case UNIONALL: {
+                UnionAllOperator unionOp = (UnionAllOperator) op;
+                if (removeUnusedVarsFromUnionAll(unionOp, toRemove)) {
+                    context.computeAndSetTypeEnvironmentForOperator(unionOp);
+                }
+                return unionOp.getVariableMappings().size();
             }
         }
         return -1;
     }
 
+    private boolean removeUnusedVarsFromUnionAll(UnionAllOperator unionOp, Set<LogicalVariable> toRemove) {
+        Iterator<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> iter = unionOp.getVariableMappings()
+                .iterator();
+        boolean modified = false;
+        Set<LogicalVariable> removeFromRemoveSet = new HashSet<LogicalVariable>();
+        while (iter.hasNext()) {
+            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping = iter.next();
+            if (toRemove.contains(varMapping.third)) {
+                iter.remove();
+                modified = true;
+            }
+            // In any case, make sure we do not removing these variables.
+            removeFromRemoveSet.add(varMapping.first);
+            removeFromRemoveSet.add(varMapping.second);
+        }
+        toRemove.removeAll(removeFromRemoveSet);
+        return modified;
+    }
+
     private boolean removeUnusedVarsAndExprs(Set<LogicalVariable> toRemove, List<LogicalVariable> varList,
             List<Mutable<ILogicalExpression>> exprList) {
         boolean changed = false;
@@ -142,22 +178,41 @@
                 }
             }
         }
-        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator assign = (AssignOperator) op;
-            toRemove.addAll(assign.getVariables());
-        } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
-            AggregateOperator agg = (AggregateOperator) op;
-            toRemove.addAll(agg.getVariables());
-        } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
-            UnnestOperator uOp = (UnnestOperator) op;
-            LogicalVariable pVar = uOp.getPositionalVariable();
-            if (pVar != null) {
-                toRemove.add(pVar);
+        boolean removeUsedVars = true;
+        switch (op.getOperatorTag()) {
+            case ASSIGN: {
+                AssignOperator assign = (AssignOperator) op;
+                toRemove.addAll(assign.getVariables());
+                break;
+            }
+            case AGGREGATE: {
+                AggregateOperator agg = (AggregateOperator) op;
+                toRemove.addAll(agg.getVariables());
+                break;
+            }
+            case UNNEST: {
+                UnnestOperator uOp = (UnnestOperator) op;
+                LogicalVariable pVar = uOp.getPositionalVariable();
+                if (pVar != null) {
+                    toRemove.add(pVar);
+                }
+                break;
+            }
+            case UNIONALL: {
+                UnionAllOperator unionOp = (UnionAllOperator) op;
+                for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping : unionOp
+                        .getVariableMappings()) {
+                    toRemove.add(varMapping.third);
+                }
+                removeUsedVars = false;
+                break;
             }
         }
-        List<LogicalVariable> used = new LinkedList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(op, used);
-        toRemove.removeAll(used);
+        if (removeUsedVars) {
+            List<LogicalVariable> used = new LinkedList<LogicalVariable>();
+            VariableUtilities.getUsedVariables(op, used);
+            toRemove.removeAll(used);
+        }
     }
 
 }
diff --git a/hivesterix/pom.xml b/hivesterix/pom.xml
index 03f22e5..cf9c4b5 100644
--- a/hivesterix/pom.xml
+++ b/hivesterix/pom.xml
@@ -102,13 +102,6 @@
 			<scope>compile</scope>

 		</dependency>

 		<dependency>

-			<groupId>org.apache.derby</groupId>

-			<artifactId>derby</artifactId>

-			<version>10.6.1.0</version>

-			<type>jar</type>

-			<scope>compile</scope>

-		</dependency>

-		<dependency>

 			<groupId>commons-collections</groupId>

 			<artifactId>commons-collections</artifactId>

 			<version>3.2.1</version>

@@ -230,13 +223,6 @@
 			<scope>compile</scope>

 		</dependency>

 		<dependency>

-			<groupId>org.apache.derby</groupId>

-			<artifactId>derby</artifactId>

-			<version>10.4.2.0</version>

-			<type>jar</type>

-			<scope>compile</scope>

-		</dependency>

-		<dependency>

 			<groupId>commons-cli</groupId>

 			<artifactId>commons-cli</artifactId>

 			<version>1.2</version>

@@ -273,13 +259,6 @@
 			<scope>compile</scope>

 		</dependency>

 		<dependency>

-			<groupId>jline</groupId>

-			<artifactId>jline</artifactId>

-			<version>0.9.94</version>

-			<type>jar</type>

-			<scope>compile</scope>

-		</dependency>

-		<dependency>

 			<groupId>org.antlr</groupId>

 			<artifactId>stringtemplate</artifactId>

 			<version>3.2</version>

@@ -336,6 +315,7 @@
 			<plugin>

 				<groupId>org.codehaus.mojo</groupId>

 				<artifactId>appassembler-maven-plugin</artifactId>

+				<version>1.3</version>

 				<executions>

 					<execution>

 						<configuration>

@@ -375,7 +355,7 @@
 			<plugin>

 				<groupId>org.apache.maven.plugins</groupId>

 				<artifactId>maven-surefire-plugin</artifactId>

-				<version>2.7.2</version>

+				<version>2.13</version>

 				<configuration>

 					<forkMode>pertest</forkMode>

 					<argLine>-enableassertions -Xmx2047m -Dfile.encoding=UTF-8 

@@ -508,6 +488,7 @@
 			</plugin>

 			<plugin>

 				<artifactId>maven-clean-plugin</artifactId>

+				<version>2.5</version>

 				<configuration>

 					<filesets>

 						<fileset>

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java
index bf4806f..3c84566 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/ExpressionConstant.java
@@ -8,18 +8,18 @@
  */

 public class ExpressionConstant {

 

-    /**

-     * name space for function identifier

-     */

-    public static String NAMESPACE = "hive";

+	/**

+	 * name space for function identifier

+	 */

+	public static String NAMESPACE = "hive";

 

-    /**

-     * field expression: modeled as function in Algebricks

-     */

-    public static String FIELDACCESS = "fieldaccess";

+	/**

+	 * field expression: modeled as function in Algebricks

+	 */

+	public static String FIELDACCESS = "fieldaccess";

 

-    /**

-     * null string: modeled as null in Algebricks

-     */

-    public static String NULL = "null";

+	/**

+	 * null string: modeled as null in Algebricks

+	 */

+	public static String NULL = "null";

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java
index 450e8ad..18380f7 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveAlgebricksBuiltInFunctionMap.java
@@ -9,73 +9,75 @@
 

 public class HiveAlgebricksBuiltInFunctionMap {

 

-    /**

-     * hive auqa builtin function map instance

-     */

-    public static HiveAlgebricksBuiltInFunctionMap INSTANCE = new HiveAlgebricksBuiltInFunctionMap();

+	/**

+	 * hive auqa builtin function map instance

+	 */

+	public static HiveAlgebricksBuiltInFunctionMap INSTANCE = new HiveAlgebricksBuiltInFunctionMap();

 

-    /**

-     * hive to Algebricks function name mapping

-     */

-    private HashMap<String, FunctionIdentifier> hiveToAlgebricksMap = new HashMap<String, FunctionIdentifier>();

+	/**

+	 * hive to Algebricks function name mapping

+	 */

+	private HashMap<String, FunctionIdentifier> hiveToAlgebricksMap = new HashMap<String, FunctionIdentifier>();

 

-    /**

-     * Algebricks to hive function name mapping

-     */

-    private HashMap<FunctionIdentifier, String> AlgebricksToHiveMap = new HashMap<FunctionIdentifier, String>();

+	/**

+	 * Algebricks to hive function name mapping

+	 */

+	private HashMap<FunctionIdentifier, String> AlgebricksToHiveMap = new HashMap<FunctionIdentifier, String>();

 

-    /**

-     * the bi-directional mapping between hive functions and Algebricks functions

-     */

-    private HiveAlgebricksBuiltInFunctionMap() {

-        hiveToAlgebricksMap.put("and", AlgebricksBuiltinFunctions.AND);

-        hiveToAlgebricksMap.put("or", AlgebricksBuiltinFunctions.OR);

-        hiveToAlgebricksMap.put("!", AlgebricksBuiltinFunctions.NOT);

-        hiveToAlgebricksMap.put("not", AlgebricksBuiltinFunctions.NOT);

-        hiveToAlgebricksMap.put("=", AlgebricksBuiltinFunctions.EQ);

-        hiveToAlgebricksMap.put("<>", AlgebricksBuiltinFunctions.NEQ);

-        hiveToAlgebricksMap.put(">", AlgebricksBuiltinFunctions.GT);

-        hiveToAlgebricksMap.put("<", AlgebricksBuiltinFunctions.LT);

-        hiveToAlgebricksMap.put(">=", AlgebricksBuiltinFunctions.GE);

-        hiveToAlgebricksMap.put("<=", AlgebricksBuiltinFunctions.LE);

+	/**

+	 * the bi-directional mapping between hive functions and Algebricks

+	 * functions

+	 */

+	private HiveAlgebricksBuiltInFunctionMap() {

+		hiveToAlgebricksMap.put("and", AlgebricksBuiltinFunctions.AND);

+		hiveToAlgebricksMap.put("or", AlgebricksBuiltinFunctions.OR);

+		hiveToAlgebricksMap.put("!", AlgebricksBuiltinFunctions.NOT);

+		hiveToAlgebricksMap.put("not", AlgebricksBuiltinFunctions.NOT);

+		hiveToAlgebricksMap.put("=", AlgebricksBuiltinFunctions.EQ);

+		hiveToAlgebricksMap.put("<>", AlgebricksBuiltinFunctions.NEQ);

+		hiveToAlgebricksMap.put(">", AlgebricksBuiltinFunctions.GT);

+		hiveToAlgebricksMap.put("<", AlgebricksBuiltinFunctions.LT);

+		hiveToAlgebricksMap.put(">=", AlgebricksBuiltinFunctions.GE);

+		hiveToAlgebricksMap.put("<=", AlgebricksBuiltinFunctions.LE);

 

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.AND, "and");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.OR, "or");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "!");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "not");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.EQ, "=");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NEQ, "<>");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GT, ">");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LT, "<");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GE, ">=");

-        AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LE, "<=");

-    }

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.AND, "and");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.OR, "or");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "!");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NOT, "not");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.EQ, "=");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.NEQ, "<>");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GT, ">");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LT, "<");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.GE, ">=");

+		AlgebricksToHiveMap.put(AlgebricksBuiltinFunctions.LE, "<=");

+	}

 

-    /**

-     * get hive function name from Algebricks function identifier

-     * 

-     * @param AlgebricksId

-     * @return hive

-     */

-    public String getHiveFunctionName(FunctionIdentifier AlgebricksId) {

-        return AlgebricksToHiveMap.get(AlgebricksId);

-    }

+	/**

+	 * get hive function name from Algebricks function identifier

+	 * 

+	 * @param AlgebricksId

+	 * @return hive

+	 */

+	public String getHiveFunctionName(FunctionIdentifier AlgebricksId) {

+		return AlgebricksToHiveMap.get(AlgebricksId);

+	}

 

-    /**

-     * get hive UDF or Generic class's corresponding built-in functions

-     * 

-     * @param funcClass

-     * @return function identifier

-     */

-    public FunctionIdentifier getAlgebricksFunctionId(Class<?> funcClass) {

-        Description annotation = (Description) funcClass.getAnnotation(Description.class);

-        String hiveUDFName = "";

-        if (annotation == null) {

-            hiveUDFName = null;

-            return null;

-        } else {

-            hiveUDFName = annotation.name();

-            return hiveToAlgebricksMap.get(hiveUDFName);

-        }

-    }

+	/**

+	 * get hive UDF or Generic class's corresponding built-in functions

+	 * 

+	 * @param funcClass

+	 * @return function identifier

+	 */

+	public FunctionIdentifier getAlgebricksFunctionId(Class<?> funcClass) {

+		Description annotation = (Description) funcClass

+				.getAnnotation(Description.class);

+		String hiveUDFName = "";

+		if (annotation == null) {

+			hiveUDFName = null;

+			return null;

+		} else {

+			hiveUDFName = annotation.name();

+			return hiveToAlgebricksMap.get(hiveUDFName);

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java
index e10e8c1..afb7d39 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveExpressionTypeComputer.java
@@ -36,144 +36,165 @@
 

 public class HiveExpressionTypeComputer implements IExpressionTypeComputer {

 

-    public static IExpressionTypeComputer INSTANCE = new HiveExpressionTypeComputer();

+	public static IExpressionTypeComputer INSTANCE = new HiveExpressionTypeComputer();

 

-    @Override

-    public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,

-            IVariableTypeEnvironment env) throws AlgebricksException {

-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {

-            /**

-             * function expression

-             */

-            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;

-            IFunctionInfo funcInfo = funcExpr.getFunctionInfo();

+	@Override

+	public Object getType(ILogicalExpression expr,

+			IMetadataProvider<?, ?> metadataProvider,

+			IVariableTypeEnvironment env) throws AlgebricksException {

+		if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {

+			/**

+			 * function expression

+			 */

+			AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;

+			IFunctionInfo funcInfo = funcExpr.getFunctionInfo();

 

-            /**

-             * argument expressions, types, object inspectors

-             */

-            List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();

-            List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();

+			/**

+			 * argument expressions, types, object inspectors

+			 */

+			List<Mutable<ILogicalExpression>> arguments = funcExpr

+					.getArguments();

+			List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();

 

-            /**

-             * get types of argument

-             */

-            for (Mutable<ILogicalExpression> argument : arguments) {

-                TypeInfo type = (TypeInfo) getType(argument.getValue(), metadataProvider, env);

-                argumentTypes.add(type);

-            }

+			/**

+			 * get types of argument

+			 */

+			for (Mutable<ILogicalExpression> argument : arguments) {

+				TypeInfo type = (TypeInfo) getType(argument.getValue(),

+						metadataProvider, env);

+				argumentTypes.add(type);

+			}

 

-            ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes.size()];

+			ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes

+					.size()];

 

-            /**

-             * get object inspector

-             */

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

-                childrenOIs[i] = TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes.get(i));

-            }

+			/**

+			 * get object inspector

+			 */

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

+				childrenOIs[i] = TypeInfoUtils

+						.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes

+								.get(i));

+			}

 

-            /**

-             * type inference for scalar function

-             */

-            if (funcExpr instanceof ScalarFunctionCallExpression) {

+			/**

+			 * type inference for scalar function

+			 */

+			if (funcExpr instanceof ScalarFunctionCallExpression) {

 

-                FunctionIdentifier AlgebricksId = funcInfo.getFunctionIdentifier();

-                Object functionInfo = ((HiveFunctionInfo) funcInfo).getInfo();

-                String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE.getHiveFunctionName(AlgebricksId);

-                GenericUDF udf;

-                if (udfName != null) {

-                    /**

-                     * get corresponding function info for built-in functions

-                     */

-                    FunctionInfo fInfo = FunctionRegistry.getFunctionInfo(udfName);

-                    udf = fInfo.getGenericUDF();

-                } else if (functionInfo != null) {

-                    /**

-                     * for GenericUDFBridge: we should not call get type of this

-                     * hive expression, because parameters may have been

-                     * changed!

-                     */

-                    ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) functionInfo;

-                    udf = hiveExpr.getGenericUDF();

-                } else {

-                    /**

-                     * for other generic UDF

-                     */

-                    Class<?> udfClass;

-                    try {

-                        udfClass = Class.forName(AlgebricksId.getName());

-                        udf = (GenericUDF) udfClass.newInstance();

-                    } catch (Exception e) {

-                        e.printStackTrace();

-                        throw new AlgebricksException(e.getMessage());

-                    }

-                }

-                /**

-                 * doing the actual type inference

-                 */

-                ObjectInspector oi = null;

-                try {

-                    oi = udf.initialize(childrenOIs);

-                } catch (Exception e) {

-                    e.printStackTrace();

-                }

+				FunctionIdentifier AlgebricksId = funcInfo

+						.getFunctionIdentifier();

+				Object functionInfo = ((HiveFunctionInfo) funcInfo).getInfo();

+				String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE

+						.getHiveFunctionName(AlgebricksId);

+				GenericUDF udf;

+				if (udfName != null) {

+					/**

+					 * get corresponding function info for built-in functions

+					 */

+					FunctionInfo fInfo = FunctionRegistry

+							.getFunctionInfo(udfName);

+					udf = fInfo.getGenericUDF();

+				} else if (functionInfo != null) {

+					/**

+					 * for GenericUDFBridge: we should not call get type of this

+					 * hive expression, because parameters may have been

+					 * changed!

+					 */

+					ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) functionInfo;

+					udf = hiveExpr.getGenericUDF();

+				} else {

+					/**

+					 * for other generic UDF

+					 */

+					Class<?> udfClass;

+					try {

+						udfClass = Class.forName(AlgebricksId.getName());

+						udf = (GenericUDF) udfClass.newInstance();

+					} catch (Exception e) {

+						e.printStackTrace();

+						throw new AlgebricksException(e.getMessage());

+					}

+				}

+				/**

+				 * doing the actual type inference

+				 */

+				ObjectInspector oi = null;

+				try {

+					oi = udf.initialize(childrenOIs);

+				} catch (Exception e) {

+					e.printStackTrace();

+				}

 

-                TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(oi);

-                return exprType;

+				TypeInfo exprType = TypeInfoUtils

+						.getTypeInfoFromObjectInspector(oi);

+				return exprType;

 

-            } else if (funcExpr instanceof AggregateFunctionCallExpression) {

-                /**

-                 * hive aggregation info

-                 */

-                AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())

-                        .getInfo();

-                /**

-                 * type inference for aggregation function

-                 */

-                GenericUDAFEvaluator result = aggregateDesc.getGenericUDAFEvaluator();

+			} else if (funcExpr instanceof AggregateFunctionCallExpression) {

+				/**

+				 * hive aggregation info

+				 */

+				AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr

+						.getFunctionInfo()).getInfo();

+				/**

+				 * type inference for aggregation function

+				 */

+				GenericUDAFEvaluator result = aggregateDesc

+						.getGenericUDAFEvaluator();

 

-                ObjectInspector returnOI = null;

-                try {

-                    returnOI = result.init(aggregateDesc.getMode(), childrenOIs);

-                } catch (HiveException e) {

-                    e.printStackTrace();

-                }

-                TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);

-                return exprType;

-            } else if (funcExpr instanceof UnnestingFunctionCallExpression) {

-                /**

-                 * type inference for UDTF function

-                 */

-                UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();

-                GenericUDTF udtf = hiveDesc.getGenericUDTF();

-                ObjectInspector returnOI = null;

-                try {

-                    returnOI = udtf.initialize(childrenOIs);

-                } catch (HiveException e) {

-                    e.printStackTrace();

-                }

-                TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);

-                return exprType;

-            } else {

-                throw new IllegalStateException("unrecognized function expression " + expr.getClass().getName());

-            }

-        } else if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-            /**

-             * get type for variable in the environment

-             */

-            VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;

-            LogicalVariable var = varExpr.getVariableReference();

-            TypeInfo type = (TypeInfo) env.getVarType(var);

-            return type;

-        } else if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {

-            /**

-             * get type for constant, from its java class

-             */

-            ConstantExpression constExpr = (ConstantExpression) expr;

-            HivesterixConstantValue value = (HivesterixConstantValue) constExpr.getValue();

-            TypeInfo type = TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(value.getObject().getClass());

-            return type;

-        } else {

-            throw new IllegalStateException("illegal expressions " + expr.getClass().getName());

-        }

-    }

+				ObjectInspector returnOI = null;

+				try {

+					returnOI = result

+							.init(aggregateDesc.getMode(), childrenOIs);

+				} catch (HiveException e) {

+					e.printStackTrace();

+				}

+				TypeInfo exprType = TypeInfoUtils

+						.getTypeInfoFromObjectInspector(returnOI);

+				return exprType;

+			} else if (funcExpr instanceof UnnestingFunctionCallExpression) {

+				/**

+				 * type inference for UDTF function

+				 */

+				UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr

+						.getFunctionInfo()).getInfo();

+				GenericUDTF udtf = hiveDesc.getGenericUDTF();

+				ObjectInspector returnOI = null;

+				try {

+					returnOI = udtf.initialize(childrenOIs);

+				} catch (HiveException e) {

+					e.printStackTrace();

+				}

+				TypeInfo exprType = TypeInfoUtils

+						.getTypeInfoFromObjectInspector(returnOI);

+				return exprType;

+			} else {

+				throw new IllegalStateException(

+						"unrecognized function expression "

+								+ expr.getClass().getName());

+			}

+		} else if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

+			/**

+			 * get type for variable in the environment

+			 */

+			VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;

+			LogicalVariable var = varExpr.getVariableReference();

+			TypeInfo type = (TypeInfo) env.getVarType(var);

+			return type;

+		} else if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {

+			/**

+			 * get type for constant, from its java class

+			 */

+			ConstantExpression constExpr = (ConstantExpression) expr;

+			HivesterixConstantValue value = (HivesterixConstantValue) constExpr

+					.getValue();

+			TypeInfo type = TypeInfoFactory

+					.getPrimitiveTypeInfoFromJavaPrimitive(value.getObject()

+							.getClass());

+			return type;

+		} else {

+			throw new IllegalStateException("illegal expressions "

+					+ expr.getClass().getName());

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java
index ced8d02..220bd00 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveFunctionInfo.java
@@ -7,30 +7,30 @@
 

 public class HiveFunctionInfo implements IFunctionInfo, Serializable {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    /**

-     * primary function identifier

-     */

-    private transient FunctionIdentifier fid;

+	/**

+	 * primary function identifier

+	 */

+	private transient FunctionIdentifier fid;

 

-    /**

-     * secondary function identifier: function name

-     */

-    private transient Object secondaryFid;

+	/**

+	 * secondary function identifier: function name

+	 */

+	private transient Object secondaryFid;

 

-    public HiveFunctionInfo(FunctionIdentifier fid, Object secondFid) {

-        this.fid = fid;

-        this.secondaryFid = secondFid;

-    }

+	public HiveFunctionInfo(FunctionIdentifier fid, Object secondFid) {

+		this.fid = fid;

+		this.secondaryFid = secondFid;

+	}

 

-    @Override

-    public FunctionIdentifier getFunctionIdentifier() {

-        return fid;

-    }

+	@Override

+	public FunctionIdentifier getFunctionIdentifier() {

+		return fid;

+	}

 

-    public Object getInfo() {

-        return secondaryFid;

-    }

+	public Object getInfo() {

+		return secondaryFid;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
index d43d7bd..8dea691 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
@@ -26,50 +26,59 @@
  * @author yingyib
  * 
  */
-public class HiveMergeAggregationExpressionFactory implements IMergeAggregationExpressionFactory {
+public class HiveMergeAggregationExpressionFactory implements
+		IMergeAggregationExpressionFactory {
 
-    public static IMergeAggregationExpressionFactory INSTANCE = new HiveMergeAggregationExpressionFactory();
+	public static IMergeAggregationExpressionFactory INSTANCE = new HiveMergeAggregationExpressionFactory();
 
-    @Override
-    public ILogicalExpression createMergeAggregation(ILogicalExpression expr, IOptimizationContext context)
-            throws AlgebricksException {
-        /**
-         * type inference for scalar function
-         */
-        if (expr instanceof AggregateFunctionCallExpression) {
-            AggregateFunctionCallExpression funcExpr = (AggregateFunctionCallExpression) expr;
-            /**
-             * hive aggregation info
-             */
-            AggregationDesc aggregator = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();
-            LogicalVariable inputVar = context.newVar();
-            ExprNodeDesc col = new ExprNodeColumnDesc(TypeInfoFactory.voidTypeInfo, inputVar.toString(), null, false);
-            ArrayList<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();
-            parameters.add(col);
+	@Override
+	public ILogicalExpression createMergeAggregation(ILogicalExpression expr,
+			IOptimizationContext context) throws AlgebricksException {
+		/**
+		 * type inference for scalar function
+		 */
+		if (expr instanceof AggregateFunctionCallExpression) {
+			AggregateFunctionCallExpression funcExpr = (AggregateFunctionCallExpression) expr;
+			/**
+			 * hive aggregation info
+			 */
+			AggregationDesc aggregator = (AggregationDesc) ((HiveFunctionInfo) funcExpr
+					.getFunctionInfo()).getInfo();
+			LogicalVariable inputVar = context.newVar();
+			ExprNodeDesc col = new ExprNodeColumnDesc(
+					TypeInfoFactory.voidTypeInfo, inputVar.toString(), null,
+					false);
+			ArrayList<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();
+			parameters.add(col);
 
-            GenericUDAFEvaluator.Mode mergeMode;
-            if (aggregator.getMode() == GenericUDAFEvaluator.Mode.PARTIAL1)
-                mergeMode = GenericUDAFEvaluator.Mode.PARTIAL2;
-            else if (aggregator.getMode() == GenericUDAFEvaluator.Mode.COMPLETE)
-                mergeMode = GenericUDAFEvaluator.Mode.FINAL;
-            else
-                mergeMode = aggregator.getMode();
-            AggregationDesc mergeDesc = new AggregationDesc(aggregator.getGenericUDAFName(),
-                    aggregator.getGenericUDAFEvaluator(), parameters, aggregator.getDistinct(), mergeMode);
+			GenericUDAFEvaluator.Mode mergeMode;
+			if (aggregator.getMode() == GenericUDAFEvaluator.Mode.PARTIAL1)
+				mergeMode = GenericUDAFEvaluator.Mode.PARTIAL2;
+			else if (aggregator.getMode() == GenericUDAFEvaluator.Mode.COMPLETE)
+				mergeMode = GenericUDAFEvaluator.Mode.FINAL;
+			else
+				mergeMode = aggregator.getMode();
+			AggregationDesc mergeDesc = new AggregationDesc(
+					aggregator.getGenericUDAFName(),
+					aggregator.getGenericUDAFEvaluator(), parameters,
+					aggregator.getDistinct(), mergeMode);
 
-            String UDAFName = mergeDesc.getGenericUDAFName();
-            List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
-            arguments.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputVar)));
+			String UDAFName = mergeDesc.getGenericUDAFName();
+			List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
+			arguments.add(new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(inputVar)));
 
-            FunctionIdentifier funcId = new FunctionIdentifier(ExpressionConstant.NAMESPACE, UDAFName + "("
-                    + mergeDesc.getMode() + ")");
-            HiveFunctionInfo funcInfo = new HiveFunctionInfo(funcId, mergeDesc);
-            AggregateFunctionCallExpression aggregationExpression = new AggregateFunctionCallExpression(funcInfo,
-                    false, arguments);
-            return aggregationExpression;
-        } else {
-            throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
-        }
-    }
+			FunctionIdentifier funcId = new FunctionIdentifier(
+					ExpressionConstant.NAMESPACE, UDAFName + "("
+							+ mergeDesc.getMode() + ")");
+			HiveFunctionInfo funcInfo = new HiveFunctionInfo(funcId, mergeDesc);
+			AggregateFunctionCallExpression aggregationExpression = new AggregateFunctionCallExpression(
+					funcInfo, false, arguments);
+			return aggregationExpression;
+		} else {
+			throw new IllegalStateException("illegal expressions "
+					+ expr.getClass().getName());
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java
index 906e3ce..10c9b8a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveNullableTypeComputer.java
@@ -5,11 +5,11 @@
 
 public class HiveNullableTypeComputer implements INullableTypeComputer {
 
-    public static INullableTypeComputer INSTANCE = new HiveNullableTypeComputer();
+	public static INullableTypeComputer INSTANCE = new HiveNullableTypeComputer();
 
-    @Override
-    public Object makeNullableType(Object type) throws AlgebricksException {
-        return type;
-    }
+	@Override
+	public Object makeNullableType(Object type) throws AlgebricksException {
+		return type;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java
index c74966c..7062e26 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivePartialAggregationTypeComputer.java
@@ -22,81 +22,95 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
-public class HivePartialAggregationTypeComputer implements IPartialAggregationTypeComputer {
+public class HivePartialAggregationTypeComputer implements
+		IPartialAggregationTypeComputer {
 
-    public static IPartialAggregationTypeComputer INSTANCE = new HivePartialAggregationTypeComputer();
+	public static IPartialAggregationTypeComputer INSTANCE = new HivePartialAggregationTypeComputer();
 
-    @Override
-    public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env,
-            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            IExpressionTypeComputer tc = HiveExpressionTypeComputer.INSTANCE;
-            /**
-             * function expression
-             */
-            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+	@Override
+	public Object getType(ILogicalExpression expr,
+			IVariableTypeEnvironment env,
+			IMetadataProvider<?, ?> metadataProvider)
+			throws AlgebricksException {
+		if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+			IExpressionTypeComputer tc = HiveExpressionTypeComputer.INSTANCE;
+			/**
+			 * function expression
+			 */
+			AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
 
-            /**
-             * argument expressions, types, object inspectors
-             */
-            List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
-            List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();
+			/**
+			 * argument expressions, types, object inspectors
+			 */
+			List<Mutable<ILogicalExpression>> arguments = funcExpr
+					.getArguments();
+			List<TypeInfo> argumentTypes = new ArrayList<TypeInfo>();
 
-            /**
-             * get types of argument
-             */
-            for (Mutable<ILogicalExpression> argument : arguments) {
-                TypeInfo type = (TypeInfo) tc.getType(argument.getValue(), metadataProvider, env);
-                argumentTypes.add(type);
-            }
+			/**
+			 * get types of argument
+			 */
+			for (Mutable<ILogicalExpression> argument : arguments) {
+				TypeInfo type = (TypeInfo) tc.getType(argument.getValue(),
+						metadataProvider, env);
+				argumentTypes.add(type);
+			}
 
-            ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes.size()];
+			ObjectInspector[] childrenOIs = new ObjectInspector[argumentTypes
+					.size()];
 
-            /**
-             * get object inspector
-             */
-            for (int i = 0; i < argumentTypes.size(); i++) {
-                childrenOIs[i] = TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes.get(i));
-            }
+			/**
+			 * get object inspector
+			 */
+			for (int i = 0; i < argumentTypes.size(); i++) {
+				childrenOIs[i] = TypeInfoUtils
+						.getStandardWritableObjectInspectorFromTypeInfo(argumentTypes
+								.get(i));
+			}
 
-            /**
-             * type inference for scalar function
-             */
-            if (funcExpr instanceof AggregateFunctionCallExpression) {
-                /**
-                 * hive aggregation info
-                 */
-                AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())
-                        .getInfo();
-                /**
-                 * type inference for aggregation function
-                 */
-                GenericUDAFEvaluator result = aggregateDesc.getGenericUDAFEvaluator();
+			/**
+			 * type inference for scalar function
+			 */
+			if (funcExpr instanceof AggregateFunctionCallExpression) {
+				/**
+				 * hive aggregation info
+				 */
+				AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr
+						.getFunctionInfo()).getInfo();
+				/**
+				 * type inference for aggregation function
+				 */
+				GenericUDAFEvaluator result = aggregateDesc
+						.getGenericUDAFEvaluator();
 
-                ObjectInspector returnOI = null;
-                try {
-                    returnOI = result.init(getPartialMode(aggregateDesc.getMode()), childrenOIs);
-                } catch (HiveException e) {
-                    e.printStackTrace();
-                }
-                TypeInfo exprType = TypeInfoUtils.getTypeInfoFromObjectInspector(returnOI);
-                return exprType;
-            } else {
-                throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
-            }
-        } else {
-            throw new IllegalStateException("illegal expressions " + expr.getClass().getName());
-        }
-    }
+				ObjectInspector returnOI = null;
+				try {
+					returnOI = result.init(
+							getPartialMode(aggregateDesc.getMode()),
+							childrenOIs);
+				} catch (HiveException e) {
+					e.printStackTrace();
+				}
+				TypeInfo exprType = TypeInfoUtils
+						.getTypeInfoFromObjectInspector(returnOI);
+				return exprType;
+			} else {
+				throw new IllegalStateException("illegal expressions "
+						+ expr.getClass().getName());
+			}
+		} else {
+			throw new IllegalStateException("illegal expressions "
+					+ expr.getClass().getName());
+		}
+	}
 
-    private Mode getPartialMode(Mode mode) {
-        Mode partialMode;
-        if (mode == Mode.FINAL)
-            partialMode = Mode.PARTIAL2;
-        else if (mode == Mode.COMPLETE)
-            partialMode = Mode.PARTIAL1;
-        else
-            partialMode = mode;
-        return partialMode;
-    }
+	private Mode getPartialMode(Mode mode) {
+		Mode partialMode;
+		if (mode == Mode.FINAL)
+			partialMode = Mode.PARTIAL2;
+		else if (mode == Mode.COMPLETE)
+			partialMode = Mode.PARTIAL1;
+		else
+			partialMode = mode;
+		return partialMode;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java
index 3d35e1f..de9cea6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/HivesterixConstantValue.java
@@ -4,52 +4,52 @@
 
 public class HivesterixConstantValue implements IAlgebricksConstantValue {
 
-    private Object object;
+	private Object object;
 
-    public HivesterixConstantValue(Object object) {
-        this.setObject(object);
-    }
+	public HivesterixConstantValue(Object object) {
+		this.setObject(object);
+	}
 
-    @Override
-    public boolean isFalse() {
-        return object == Boolean.FALSE;
-    }
+	@Override
+	public boolean isFalse() {
+		return object == Boolean.FALSE;
+	}
 
-    @Override
-    public boolean isNull() {
-        return object == null;
-    }
+	@Override
+	public boolean isNull() {
+		return object == null;
+	}
 
-    @Override
-    public boolean isTrue() {
-        return object == Boolean.TRUE;
-    }
+	@Override
+	public boolean isTrue() {
+		return object == Boolean.TRUE;
+	}
 
-    public void setObject(Object object) {
-        this.object = object;
-    }
+	public void setObject(Object object) {
+		this.object = object;
+	}
 
-    public Object getObject() {
-        return object;
-    }
+	public Object getObject() {
+		return object;
+	}
 
-    @Override
-    public String toString() {
-        return object.toString();
-    }
+	@Override
+	public String toString() {
+		return object.toString();
+	}
 
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof HivesterixConstantValue)) {
-            return false;
-        }
-        HivesterixConstantValue v2 = (HivesterixConstantValue) o;
-        return object.equals(v2.getObject());
-    }
+	@Override
+	public boolean equals(Object o) {
+		if (!(o instanceof HivesterixConstantValue)) {
+			return false;
+		}
+		HivesterixConstantValue v2 = (HivesterixConstantValue) o;
+		return object.equals(v2.getObject());
+	}
 
-    @Override
-    public int hashCode() {
-        return object.hashCode();
-    }
+	@Override
+	public int hashCode() {
+		return object.hashCode();
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java
index 3982d33..2b1d191 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/expression/Schema.java
@@ -10,30 +10,30 @@
 

 public class Schema implements Serializable {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private List<String> fieldNames;

+	private List<String> fieldNames;

 

-    private List<TypeInfo> fieldTypes;

+	private List<TypeInfo> fieldTypes;

 

-    public Schema(List<String> fieldNames, List<TypeInfo> fieldTypes) {

-        this.fieldNames = fieldNames;

-        this.fieldTypes = fieldTypes;

-    }

+	public Schema(List<String> fieldNames, List<TypeInfo> fieldTypes) {

+		this.fieldNames = fieldNames;

+		this.fieldTypes = fieldTypes;

+	}

 

-    public ObjectInspector toObjectInspector() {

-        return LazyUtils.getLazyObjectInspector(fieldNames, fieldTypes);

-    }

+	public ObjectInspector toObjectInspector() {

+		return LazyUtils.getLazyObjectInspector(fieldNames, fieldTypes);

+	}

 

-    public List<String> getNames() {

-        return fieldNames;

-    }

+	public List<String> getNames() {

+		return fieldNames;

+	}

 

-    public List<TypeInfo> getTypes() {

-        return fieldTypes;

-    }

+	public List<TypeInfo> getTypes() {

+		return fieldTypes;

+	}

 

-    public Object[] getSchema() {

-        return fieldTypes.toArray();

-    }

+	public Object[] getSchema() {

+		return fieldTypes.toArray();

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
index 7586d83..1fb973e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveAlgebricksTranslator.java
@@ -77,6 +77,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;

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

 

+@SuppressWarnings("rawtypes")

 public class HiveAlgebricksTranslator implements Translator {

 

     private int currentVariable = 0;

@@ -212,6 +213,7 @@
 

     /**

      * get the number of variables

+     * s

      * 

      * @return

      */

@@ -373,6 +375,7 @@
                 ;

             if (hiveOperator.getChildOperators() != null && hiveOperator.getChildOperators().size() > 0

                     && continueTraverse) {

+                @SuppressWarnings("unchecked")

                 List<Operator> children = hiveOperator.getChildOperators();

                 if (currentOperatorRef == null)

                     currentOperatorRef = AlgebricksParentOperator;

@@ -521,8 +524,8 @@
         printOperatorSchema(operator);

         List<ColumnInfo> columns = operator.getSchema().getSignature();

         if (variables.size() != columns.size()) {

-            System.out.println("output cardinality error " + operator.getName() + " variable size: " + variables.size()

-                    + " expected " + columns.size());

+            throw new IllegalStateException("output cardinality error " + operator.getName() + " variable size: "

+                    + variables.size() + " expected " + columns.size());

         }

 

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

@@ -556,7 +559,7 @@
                         fieldName = "null." + desc.getColumn();

                         var = getVariableOnly(fieldName);

                         if (var == null) {

-                            System.out.println(fieldName + " is wrong!!! ");

+                            throw new IllegalStateException(fieldName + " is wrong!!! ");

                         }

                     }

                 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java
index a34371a..494e796 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveLogicalPlanAndMetaData.java
@@ -5,30 +5,32 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;

 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;

 

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

 public class HiveLogicalPlanAndMetaData implements ILogicalPlanAndMetadata {

 

-    IMetadataProvider metadata;

-    ILogicalPlan plan;

+	IMetadataProvider metadata;

+	ILogicalPlan plan;

 

-    public HiveLogicalPlanAndMetaData(ILogicalPlan plan, IMetadataProvider metadata) {

-        this.plan = plan;

-        this.metadata = metadata;

-    }

+	public HiveLogicalPlanAndMetaData(ILogicalPlan plan,

+			IMetadataProvider metadata) {

+		this.plan = plan;

+		this.metadata = metadata;

+	}

 

-    @Override

-    public IMetadataProvider getMetadataProvider() {

-        return metadata;

-    }

+	@Override

+	public IMetadataProvider getMetadataProvider() {

+		return metadata;

+	}

 

-    @Override

-    public ILogicalPlan getPlan() {

-        return plan;

-    }

+	@Override

+	public ILogicalPlan getPlan() {

+		return plan;

+	}

 

-    @Override

-    public AlgebricksPartitionConstraint getClusterLocations() {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public AlgebricksPartitionConstraint getClusterLocations() {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java
index bf59900..0d234fb 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/HiveOperatorAnnotations.java
@@ -2,7 +2,7 @@
 
 public class HiveOperatorAnnotations {
 
-    // hints
-    public static final String LOCAL_GROUP_BY = "LOCAL_GROUP_BY"; 
-    
+	// hints
+	public static final String LOCAL_GROUP_BY = "LOCAL_GROUP_BY";
+
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java
index 6b0aacd..9a84164 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ExtractVisitor.java
@@ -13,14 +13,16 @@
 

 public class ExtractVisitor extends DefaultVisitor {

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ExtractOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) {

-        Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));

-        operator.setSchema(operator.getParentOperators().get(0).getSchema());

-        List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);

-        t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(ExtractOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		Schema currentSchema = t.generateInputSchema(operator

+				.getParentOperators().get(0));

+		operator.setSchema(operator.getParentOperators().get(0).getSchema());

+		List<LogicalVariable> latestOutputSchema = t

+				.getVariablesFromSchema(currentSchema);

+		t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
index 529fc09..b276ba9 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/FilterVisitor.java
@@ -18,24 +18,27 @@
 

 public class FilterVisitor extends DefaultVisitor {

 

-    @Override

-    public Mutable<ILogicalOperator> visit(FilterOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) {

-        Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));

+	@Override

+	public Mutable<ILogicalOperator> visit(FilterOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		Schema currentSchema = t.generateInputSchema(operator

+				.getParentOperators().get(0));

 

-        FilterDesc desc = (FilterDesc) operator.getConf();

-        ExprNodeDesc predicate = desc.getPredicate();

-        t.rewriteExpression(predicate);

+		FilterDesc desc = (FilterDesc) operator.getConf();

+		ExprNodeDesc predicate = desc.getPredicate();

+		t.rewriteExpression(predicate);

 

-        Mutable<ILogicalExpression> exprs = t.translateScalarFucntion(desc.getPredicate());

-        ILogicalOperator currentOperator = new SelectOperator(exprs);

-        currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+		Mutable<ILogicalExpression> exprs = t.translateScalarFucntion(desc

+				.getPredicate());

+		ILogicalOperator currentOperator = new SelectOperator(exprs);

+		currentOperator.getInputs().add(AlgebricksParentOperatorRef);

 

-        // populate the schema from upstream operator

-        operator.setSchema(operator.getParentOperators().get(0).getSchema());

-        List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);

-        t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		// populate the schema from upstream operator

+		operator.setSchema(operator.getParentOperators().get(0).getSchema());

+		List<LogicalVariable> latestOutputSchema = t

+				.getVariablesFromSchema(currentSchema);

+		t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java
index 4f6a31f..d2180a3 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/GroupByVisitor.java
@@ -44,220 +44,248 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;

 

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

 public class GroupByVisitor extends DefaultVisitor {

 

-    private List<Mutable<ILogicalExpression>> AlgebricksAggs = new ArrayList<Mutable<ILogicalExpression>>();

-    private List<IFunctionInfo> localAggs = new ArrayList<IFunctionInfo>();

-    private boolean isDistinct = false;

-    private boolean gbyKeyNotRedKey = false;

+	private List<Mutable<ILogicalExpression>> AlgebricksAggs = new ArrayList<Mutable<ILogicalExpression>>();

+	private List<IFunctionInfo> localAggs = new ArrayList<IFunctionInfo>();

+	private boolean isDistinct = false;

+	private boolean gbyKeyNotRedKey = false;

 

-    @Override

-    public Mutable<ILogicalOperator> visit(GroupByOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {

+	@Override

+	public Mutable<ILogicalOperator> visit(GroupByOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException {

 

-        // get descriptors

-        GroupByDesc desc = (GroupByDesc) operator.getConf();

-        GroupByDesc.Mode mode = desc.getMode();

+		// get descriptors

+		GroupByDesc desc = (GroupByDesc) operator.getConf();

+		GroupByDesc.Mode mode = desc.getMode();

 

-        List<ExprNodeDesc> keys = desc.getKeys();

-        List<AggregationDesc> aggregators = desc.getAggregators();

+		List<ExprNodeDesc> keys = desc.getKeys();

+		List<AggregationDesc> aggregators = desc.getAggregators();

 

-        Operator child = operator.getChildOperators().get(0);

+		Operator child = operator.getChildOperators().get(0);

 

-        if (child.getType() == OperatorType.REDUCESINK) {

-            List<ExprNodeDesc> partKeys = ((ReduceSinkDesc) child.getConf()).getPartitionCols();

-            if (keys.size() != partKeys.size())

-                gbyKeyNotRedKey = true;

-        }

+		if (child.getType() == OperatorType.REDUCESINK) {

+			List<ExprNodeDesc> partKeys = ((ReduceSinkDesc) child.getConf())

+					.getPartitionCols();

+			if (keys.size() != partKeys.size())

+				gbyKeyNotRedKey = true;

+		}

 

-        if (mode == GroupByDesc.Mode.PARTIAL1 || mode == GroupByDesc.Mode.HASH || mode == GroupByDesc.Mode.COMPLETE

-                || (aggregators.size() == 0 && isDistinct == false) || gbyKeyNotRedKey) {

-            AlgebricksAggs.clear();

-            // add an assign operator if the key is not a column expression

-            ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

-            ILogicalOperator currentOperator = null;

-            ILogicalOperator assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);

-            if (assignOperator != null) {

-                currentOperator = assignOperator;

-                AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-            }

+		if (mode == GroupByDesc.Mode.PARTIAL1 || mode == GroupByDesc.Mode.HASH

+				|| mode == GroupByDesc.Mode.COMPLETE

+				|| (aggregators.size() == 0 && isDistinct == false)

+				|| gbyKeyNotRedKey) {

+			AlgebricksAggs.clear();

+			// add an assign operator if the key is not a column expression

+			ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

+			ILogicalOperator currentOperator = null;

+			ILogicalOperator assignOperator = t.getAssignOperator(

+					AlgebricksParentOperatorRef, keys, keyVariables);

+			if (assignOperator != null) {

+				currentOperator = assignOperator;

+				AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+						currentOperator);

+			}

 

-            // get key variable expression list

-            List<Mutable<ILogicalExpression>> keyExprs = new ArrayList<Mutable<ILogicalExpression>>();

-            for (LogicalVariable var : keyVariables) {

-                keyExprs.add(t.translateScalarFucntion(new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, var

-                        .toString(), "", false)));

-            }

+			// get key variable expression list

+			List<Mutable<ILogicalExpression>> keyExprs = new ArrayList<Mutable<ILogicalExpression>>();

+			for (LogicalVariable var : keyVariables) {

+				keyExprs.add(t.translateScalarFucntion(new ExprNodeColumnDesc(

+						TypeInfoFactory.intTypeInfo, var.toString(), "", false)));

+			}

 

-            if (aggregators.size() == 0) {

-                List<Mutable<ILogicalExpression>> distinctExprs = new ArrayList<Mutable<ILogicalExpression>>();

-                for (LogicalVariable var : keyVariables) {

-                    Mutable<ILogicalExpression> varExpr = new MutableObject<ILogicalExpression>(

-                            new VariableReferenceExpression(var));

-                    distinctExprs.add(varExpr);

-                }

-                t.rewriteOperatorOutputSchema(keyVariables, operator);

-                isDistinct = true;

-                ILogicalOperator lop = new DistinctOperator(distinctExprs);

-                lop.getInputs().add(AlgebricksParentOperatorRef);

-                return new MutableObject<ILogicalOperator>(lop);

-            }

+			if (aggregators.size() == 0) {

+				List<Mutable<ILogicalExpression>> distinctExprs = new ArrayList<Mutable<ILogicalExpression>>();

+				for (LogicalVariable var : keyVariables) {

+					Mutable<ILogicalExpression> varExpr = new MutableObject<ILogicalExpression>(

+							new VariableReferenceExpression(var));

+					distinctExprs.add(varExpr);

+				}

+				t.rewriteOperatorOutputSchema(keyVariables, operator);

+				isDistinct = true;

+				ILogicalOperator lop = new DistinctOperator(distinctExprs);

+				lop.getInputs().add(AlgebricksParentOperatorRef);

+				return new MutableObject<ILogicalOperator>(lop);

+			}

 

-            // get the pair<LogicalVariable, ILogicalExpression> list

-            List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> keyParameters = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();

-            keyVariables.clear();

-            for (Mutable<ILogicalExpression> expr : keyExprs) {

-                LogicalVariable keyVar = t.getVariable(expr.getValue().toString(), TypeInfoFactory.unknownTypeInfo);

-                keyParameters.add(new Pair(keyVar, expr));

-                keyVariables.add(keyVar);

-            }

+			// get the pair<LogicalVariable, ILogicalExpression> list

+			List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> keyParameters = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();

+			keyVariables.clear();

+			for (Mutable<ILogicalExpression> expr : keyExprs) {

+				LogicalVariable keyVar = t.getVariable(expr.getValue()

+						.toString(), TypeInfoFactory.unknownTypeInfo);

+				keyParameters.add(new Pair(keyVar, expr));

+				keyVariables.add(keyVar);

+			}

 

-            // get the parameters for the aggregator operator

-            ArrayList<LogicalVariable> aggVariables = new ArrayList<LogicalVariable>();

-            ArrayList<Mutable<ILogicalExpression>> aggExprs = new ArrayList<Mutable<ILogicalExpression>>();

+			// get the parameters for the aggregator operator

+			ArrayList<LogicalVariable> aggVariables = new ArrayList<LogicalVariable>();

+			ArrayList<Mutable<ILogicalExpression>> aggExprs = new ArrayList<Mutable<ILogicalExpression>>();

 

-            // get the type of each aggregation function

-            HashMap<AggregationDesc, TypeInfo> aggToType = new HashMap<AggregationDesc, TypeInfo>();

-            List<ColumnInfo> columns = operator.getSchema().getSignature();

-            int offset = keys.size();

-            for (int i = offset; i < columns.size(); i++) {

-                aggToType.put(aggregators.get(i - offset), columns.get(i).getType());

-            }

+			// get the type of each aggregation function

+			HashMap<AggregationDesc, TypeInfo> aggToType = new HashMap<AggregationDesc, TypeInfo>();

+			List<ColumnInfo> columns = operator.getSchema().getSignature();

+			int offset = keys.size();

+			for (int i = offset; i < columns.size(); i++) {

+				aggToType.put(aggregators.get(i - offset), columns.get(i)

+						.getType());

+			}

 

-            localAggs.clear();

-            // rewrite parameter expressions for all aggregators

-            for (AggregationDesc aggregator : aggregators) {

-                for (ExprNodeDesc parameter : aggregator.getParameters()) {

-                    t.rewriteExpression(parameter);

-                }

-                Mutable<ILogicalExpression> aggExpr = t.translateAggregation(aggregator);

-                AbstractFunctionCallExpression localAggExpr = (AbstractFunctionCallExpression) aggExpr.getValue();

-                localAggs.add(localAggExpr.getFunctionInfo());

+			localAggs.clear();

+			// rewrite parameter expressions for all aggregators

+			for (AggregationDesc aggregator : aggregators) {

+				for (ExprNodeDesc parameter : aggregator.getParameters()) {

+					t.rewriteExpression(parameter);

+				}

+				Mutable<ILogicalExpression> aggExpr = t

+						.translateAggregation(aggregator);

+				AbstractFunctionCallExpression localAggExpr = (AbstractFunctionCallExpression) aggExpr

+						.getValue();

+				localAggs.add(localAggExpr.getFunctionInfo());

 

-                AggregationDesc logicalAgg = new AggregationDesc(aggregator.getGenericUDAFName(),

-                        aggregator.getGenericUDAFEvaluator(), aggregator.getParameters(), aggregator.getDistinct(),

-                        Mode.COMPLETE);

-                Mutable<ILogicalExpression> logicalAggExpr = t.translateAggregation(logicalAgg);

+				AggregationDesc logicalAgg = new AggregationDesc(

+						aggregator.getGenericUDAFName(),

+						aggregator.getGenericUDAFEvaluator(),

+						aggregator.getParameters(), aggregator.getDistinct(),

+						Mode.COMPLETE);

+				Mutable<ILogicalExpression> logicalAggExpr = t

+						.translateAggregation(logicalAgg);

 

-                AlgebricksAggs.add(logicalAggExpr);

-                if (!gbyKeyNotRedKey)

-                    aggExprs.add(logicalAggExpr);

-                else

-                    aggExprs.add(aggExpr);

+				AlgebricksAggs.add(logicalAggExpr);

+				if (!gbyKeyNotRedKey)

+					aggExprs.add(logicalAggExpr);

+				else

+					aggExprs.add(aggExpr);

 

-                aggVariables.add(t.getVariable(aggregator.getExprString() + aggregator.getMode(),

-                        aggToType.get(aggregator)));

-            }

+				aggVariables.add(t.getVariable(aggregator.getExprString()

+						+ aggregator.getMode(), aggToType.get(aggregator)));

+			}

 

-            if (child.getType() != OperatorType.REDUCESINK)

-                gbyKeyNotRedKey = false;

+			if (child.getType() != OperatorType.REDUCESINK)

+				gbyKeyNotRedKey = false;

 

-            // get the sub plan list

-            AggregateOperator aggOperator = new AggregateOperator(aggVariables, aggExprs);

-            NestedTupleSourceOperator nestedTupleSource = new NestedTupleSourceOperator(

-                    new MutableObject<ILogicalOperator>());

-            aggOperator.getInputs().add(new MutableObject<ILogicalOperator>(nestedTupleSource));

+			// get the sub plan list

+			AggregateOperator aggOperator = new AggregateOperator(aggVariables,

+					aggExprs);

+			NestedTupleSourceOperator nestedTupleSource = new NestedTupleSourceOperator(

+					new MutableObject<ILogicalOperator>());

+			aggOperator.getInputs().add(

+					new MutableObject<ILogicalOperator>(nestedTupleSource));

 

-            List<Mutable<ILogicalOperator>> subRoots = new ArrayList<Mutable<ILogicalOperator>>();

-            subRoots.add(new MutableObject<ILogicalOperator>(aggOperator));

-            ILogicalPlan subPlan = new ALogicalPlanImpl(subRoots);

-            List<ILogicalPlan> subPlans = new ArrayList<ILogicalPlan>();

-            subPlans.add(subPlan);

+			List<Mutable<ILogicalOperator>> subRoots = new ArrayList<Mutable<ILogicalOperator>>();

+			subRoots.add(new MutableObject<ILogicalOperator>(aggOperator));

+			ILogicalPlan subPlan = new ALogicalPlanImpl(subRoots);

+			List<ILogicalPlan> subPlans = new ArrayList<ILogicalPlan>();

+			subPlans.add(subPlan);

 

-            // create the group by operator

-            currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator(

-                    keyParameters, new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>(), subPlans);

-            currentOperator.getInputs().add(AlgebricksParentOperatorRef);

-            nestedTupleSource.getDataSourceReference().setValue(currentOperator);

+			// create the group by operator

+			currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator(

+					keyParameters,

+					new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>(),

+					subPlans);

+			currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+			nestedTupleSource.getDataSourceReference()

+					.setValue(currentOperator);

 

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

-            outputVariables.addAll(keyVariables);

-            outputVariables.addAll(aggVariables);

-            t.rewriteOperatorOutputSchema(outputVariables, operator);

+			List<LogicalVariable> outputVariables = new ArrayList<LogicalVariable>();

+			outputVariables.addAll(keyVariables);

+			outputVariables.addAll(aggVariables);

+			t.rewriteOperatorOutputSchema(outputVariables, operator);

 

-            if (gbyKeyNotRedKey) {

-                currentOperator.getAnnotations().put(HiveOperatorAnnotations.LOCAL_GROUP_BY, Boolean.TRUE);

-            }

+			if (gbyKeyNotRedKey) {

+				currentOperator.getAnnotations().put(

+						HiveOperatorAnnotations.LOCAL_GROUP_BY, Boolean.TRUE);

+			}

 

-            HiveConf conf = ConfUtil.getHiveConf();

-            Boolean extGby = conf.getBoolean("hive.algebricks.groupby.external", false);

+			HiveConf conf = ConfUtil.getHiveConf();

+			Boolean extGby = conf.getBoolean(

+					"hive.algebricks.groupby.external", false);

 

-            if (extGby && isSerializable(aggregators)) {

-                currentOperator.getAnnotations().put(OperatorAnnotations.USE_EXTERNAL_GROUP_BY, Boolean.TRUE);

-            }

-            return new MutableObject<ILogicalOperator>(currentOperator);

-        } else {

-            isDistinct = false;

-            // rewrite parameter expressions for all aggregators

-            int i = 0;

-            for (AggregationDesc aggregator : aggregators) {

-                for (ExprNodeDesc parameter : aggregator.getParameters()) {

-                    t.rewriteExpression(parameter);

-                }

-                Mutable<ILogicalExpression> agg = t.translateAggregation(aggregator);

-                AggregateFunctionCallExpression originalAgg = (AggregateFunctionCallExpression) AlgebricksAggs.get(i)

-                        .getValue();

-                originalAgg.setStepOneAggregate(localAggs.get(i));

-                AggregateFunctionCallExpression currentAgg = (AggregateFunctionCallExpression) agg.getValue();

-                if (currentAgg.getFunctionInfo() != null) {

-                    originalAgg.setTwoStep(true);

-                    originalAgg.setStepTwoAggregate(currentAgg.getFunctionInfo());

-                }

-                i++;

-            }

-            return null;

-        }

-    }

+			if (extGby && isSerializable(aggregators)) {

+				currentOperator.getAnnotations()

+						.put(OperatorAnnotations.USE_EXTERNAL_GROUP_BY,

+								Boolean.TRUE);

+			}

+			return new MutableObject<ILogicalOperator>(currentOperator);

+		} else {

+			isDistinct = false;

+			// rewrite parameter expressions for all aggregators

+			int i = 0;

+			for (AggregationDesc aggregator : aggregators) {

+				for (ExprNodeDesc parameter : aggregator.getParameters()) {

+					t.rewriteExpression(parameter);

+				}

+				Mutable<ILogicalExpression> agg = t

+						.translateAggregation(aggregator);

+				AggregateFunctionCallExpression originalAgg = (AggregateFunctionCallExpression) AlgebricksAggs

+						.get(i).getValue();

+				originalAgg.setStepOneAggregate(localAggs.get(i));

+				AggregateFunctionCallExpression currentAgg = (AggregateFunctionCallExpression) agg

+						.getValue();

+				if (currentAgg.getFunctionInfo() != null) {

+					originalAgg.setTwoStep(true);

+					originalAgg.setStepTwoAggregate(currentAgg

+							.getFunctionInfo());

+				}

+				i++;

+			}

+			return null;

+		}

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

-        Operator downStream = (Operator) operator.getChildOperators().get(0);

-        if (!(downStream instanceof GroupByOperator)) {

-            return null;

-        }

+	@Override

+	public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		Operator downStream = (Operator) operator.getChildOperators().get(0);

+		if (!(downStream instanceof GroupByOperator)) {

+			return null;

+		}

 

-        ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

-        List<ExprNodeDesc> keys = desc.getKeyCols();

-        List<ExprNodeDesc> values = desc.getValueCols();

+		ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

+		List<ExprNodeDesc> keys = desc.getKeyCols();

+		List<ExprNodeDesc> values = desc.getValueCols();

 

-        // insert assign for keys

-        ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

-        t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);

+		// insert assign for keys

+		ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

+		t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);

 

-        // insert assign for values

-        ArrayList<LogicalVariable> valueVariables = new ArrayList<LogicalVariable>();

-        t.getAssignOperator(AlgebricksParentOperatorRef, values, valueVariables);

+		// insert assign for values

+		ArrayList<LogicalVariable> valueVariables = new ArrayList<LogicalVariable>();

+		t.getAssignOperator(AlgebricksParentOperatorRef, values, valueVariables);

 

-        ArrayList<LogicalVariable> columns = new ArrayList<LogicalVariable>();

-        columns.addAll(keyVariables);

-        columns.addAll(valueVariables);

+		ArrayList<LogicalVariable> columns = new ArrayList<LogicalVariable>();

+		columns.addAll(keyVariables);

+		columns.addAll(valueVariables);

 

-        t.rewriteOperatorOutputSchema(columns, operator);

-        return null;

-    }

+		t.rewriteOperatorOutputSchema(columns, operator);

+		return null;

+	}

 

-    private boolean isSerializable(List<AggregationDesc> descs) throws AlgebricksException {

-        try {

-            for (AggregationDesc desc : descs) {

-                GenericUDAFEvaluator udaf = desc.getGenericUDAFEvaluator();

-                AggregationBuffer buf = udaf.getNewAggregationBuffer();

-                Class<?> bufferClass = buf.getClass();

-                Field[] fields = bufferClass.getDeclaredFields();

-                for (Field field : fields) {

-                    field.setAccessible(true);

-                    String type = field.getType().toString();

-                    if (!(type.equals("int") || type.equals("long") || type.equals("float") || type.equals("double") || type

-                            .equals("boolean"))) {

-                        return false;

-                    }

-                }

+	private boolean isSerializable(List<AggregationDesc> descs)

+			throws AlgebricksException {

+		try {

+			for (AggregationDesc desc : descs) {

+				GenericUDAFEvaluator udaf = desc.getGenericUDAFEvaluator();

+				AggregationBuffer buf = udaf.getNewAggregationBuffer();

+				Class<?> bufferClass = buf.getClass();

+				Field[] fields = bufferClass.getDeclaredFields();

+				for (Field field : fields) {

+					field.setAccessible(true);

+					String type = field.getType().toString();

+					if (!(type.equals("int") || type.equals("long")

+							|| type.equals("float") || type.equals("double") || type

+								.equals("boolean"))) {

+						return false;

+					}

+				}

 

-            }

-            return true;

-        } catch (Exception e) {

-            throw new AlgebricksException(e);

-        }

-    }

+			}

+			return true;

+		} catch (Exception e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java
index 41a492d..aea4be5 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/JoinVisitor.java
@@ -33,382 +33,413 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;

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

 

+@SuppressWarnings("rawtypes")

 public class JoinVisitor extends DefaultVisitor {

 

-    /**

-     * reduce sink operator to variables

-     */

-    private HashMap<Operator, List<LogicalVariable>> reduceSinkToKeyVariables = new HashMap<Operator, List<LogicalVariable>>();

+	/**

+	 * reduce sink operator to variables

+	 */

+	private HashMap<Operator, List<LogicalVariable>> reduceSinkToKeyVariables = new HashMap<Operator, List<LogicalVariable>>();

 

-    /**

-     * reduce sink operator to variables

-     */

-    private HashMap<Operator, List<String>> reduceSinkToFieldNames = new HashMap<Operator, List<String>>();

+	/**

+	 * reduce sink operator to variables

+	 */

+	private HashMap<Operator, List<String>> reduceSinkToFieldNames = new HashMap<Operator, List<String>>();

 

-    /**

-     * reduce sink operator to variables

-     */

-    private HashMap<Operator, List<TypeInfo>> reduceSinkToTypes = new HashMap<Operator, List<TypeInfo>>();

+	/**

+	 * reduce sink operator to variables

+	 */

+	private HashMap<Operator, List<TypeInfo>> reduceSinkToTypes = new HashMap<Operator, List<TypeInfo>>();

 

-    /**

-     * map a join operator (in hive) to its parent operators (in hive)

-     */

-    private HashMap<Operator, List<Operator>> operatorToHiveParents = new HashMap<Operator, List<Operator>>();

+	/**

+	 * map a join operator (in hive) to its parent operators (in hive)

+	 */

+	private HashMap<Operator, List<Operator>> operatorToHiveParents = new HashMap<Operator, List<Operator>>();

 

-    /**

-     * map a join operator (in hive) to its parent operators (in asterix)

-     */

-    private HashMap<Operator, List<ILogicalOperator>> operatorToAsterixParents = new HashMap<Operator, List<ILogicalOperator>>();

+	/**

+	 * map a join operator (in hive) to its parent operators (in asterix)

+	 */

+	private HashMap<Operator, List<ILogicalOperator>> operatorToAsterixParents = new HashMap<Operator, List<ILogicalOperator>>();

 

-    /**

-     * the latest traversed reduce sink operator

-     */

-    private Operator latestReduceSink = null;

+	/**

+	 * the latest traversed reduce sink operator

+	 */

+	private Operator latestReduceSink = null;

 

-    /**

-     * the latest generated parent for join

-     */

-    private ILogicalOperator latestAlgebricksOperator = null;

+	/**

+	 * the latest generated parent for join

+	 */

+	private ILogicalOperator latestAlgebricksOperator = null;

 

-    /**

-     * process a join operator

-     */

-    @Override

-    public Mutable<ILogicalOperator> visit(JoinOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) {

-        latestAlgebricksOperator = AlgebricksParentOperator.getValue();

-        translateJoinOperatorPreprocess(operator, t);

-        List<Operator> parents = operatorToHiveParents.get(operator);

-        if (parents.size() < operator.getParentOperators().size()) {

-            return null;

-        } else {

-            ILogicalOperator joinOp = translateJoinOperator(operator, AlgebricksParentOperator, t);

-            // clearStatus();

-            return new MutableObject<ILogicalOperator>(joinOp);

-        }

-    }

+	/**

+	 * process a join operator

+	 */

+	@Override

+	public Mutable<ILogicalOperator> visit(JoinOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) {

+		latestAlgebricksOperator = AlgebricksParentOperator.getValue();

+		translateJoinOperatorPreprocess(operator, t);

+		List<Operator> parents = operatorToHiveParents.get(operator);

+		if (parents.size() < operator.getParentOperators().size()) {

+			return null;

+		} else {

+			ILogicalOperator joinOp = translateJoinOperator(operator,

+					AlgebricksParentOperator, t);

+			// clearStatus();

+			return new MutableObject<ILogicalOperator>(joinOp);

+		}

+	}

 

-    private void reorder(Byte[] order, List<ILogicalOperator> parents, List<Operator> hiveParents) {

-        ILogicalOperator[] lops = new ILogicalOperator[parents.size()];

-        Operator[] ops = new Operator[hiveParents.size()];

+	private void reorder(Byte[] order, List<ILogicalOperator> parents,

+			List<Operator> hiveParents) {

+		ILogicalOperator[] lops = new ILogicalOperator[parents.size()];

+		Operator[] ops = new Operator[hiveParents.size()];

 

-        for (Operator op : hiveParents) {

-            ReduceSinkOperator rop = (ReduceSinkOperator) op;

-            ReduceSinkDesc rdesc = rop.getConf();

-            int tag = rdesc.getTag();

+		for (Operator op : hiveParents) {

+			ReduceSinkOperator rop = (ReduceSinkOperator) op;

+			ReduceSinkDesc rdesc = rop.getConf();

+			int tag = rdesc.getTag();

 

-            int index = -1;

-            for (int i = 0; i < order.length; i++)

-                if (order[i] == tag) {

-                    index = i;

-                    break;

-                }

-            lops[index] = parents.get(hiveParents.indexOf(op));

-            ops[index] = op;

-        }

+			int index = -1;

+			for (int i = 0; i < order.length; i++)

+				if (order[i] == tag) {

+					index = i;

+					break;

+				}

+			lops[index] = parents.get(hiveParents.indexOf(op));

+			ops[index] = op;

+		}

 

-        parents.clear();

-        hiveParents.clear();

+		parents.clear();

+		hiveParents.clear();

 

-        for (int i = 0; i < lops.length; i++) {

-            parents.add(lops[i]);

-            hiveParents.add(ops[i]);

-        }

-    }

+		for (int i = 0; i < lops.length; i++) {

+			parents.add(lops[i]);

+			hiveParents.add(ops[i]);

+		}

+	}

 

-    /**

-     * translate a hive join operator to asterix join operator->assign

-     * operator->project operator

-     * 

-     * @param parentOperator

-     * @param operator

-     * @return

-     */

-    private ILogicalOperator translateJoinOperator(Operator operator, Mutable<ILogicalOperator> parentOperator,

-            Translator t) {

+	/**

+	 * translate a hive join operator to asterix join operator->assign

+	 * operator->project operator

+	 * 

+	 * @param parentOperator

+	 * @param operator

+	 * @return

+	 */

+	private ILogicalOperator translateJoinOperator(Operator operator,

+			Mutable<ILogicalOperator> parentOperator, Translator t) {

 

-        JoinDesc joinDesc = (JoinDesc) operator.getConf();

+		JoinDesc joinDesc = (JoinDesc) operator.getConf();

 

-        // get the projection expression (already re-written) from each source

-        // table

-        Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();

-        reorder(joinDesc.getTagOrder(), operatorToAsterixParents.get(operator), operatorToHiveParents.get(operator));

+		// get the projection expression (already re-written) from each source

+		// table

+		Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();

+		reorder(joinDesc.getTagOrder(), operatorToAsterixParents.get(operator),

+				operatorToHiveParents.get(operator));

 

-        // make an reduce join operator

-        ILogicalOperator currentOperator = generateJoinTree(joinDesc.getCondsList(),

-                operatorToAsterixParents.get(operator), operatorToHiveParents.get(operator), 0, t);

-        parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

+		// make an reduce join operator

+		ILogicalOperator currentOperator = generateJoinTree(

+				joinDesc.getCondsList(),

+				operatorToAsterixParents.get(operator),

+				operatorToHiveParents.get(operator), 0, t);

+		parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

 

-        // add assign and project operator on top of a join

-        // output variables

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

-        Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();

-        Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();

-        while (iterator.hasNext()) {

-            List<ExprNodeDesc> outputExprs = iterator.next().getValue();

-            ILogicalOperator assignOperator = t.getAssignOperator(parentOperator, outputExprs, variables);

+		// add assign and project operator on top of a join

+		// output variables

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();

+		Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();

+		while (iterator.hasNext()) {

+			List<ExprNodeDesc> outputExprs = iterator.next().getValue();

+			ILogicalOperator assignOperator = t.getAssignOperator(

+					parentOperator, outputExprs, variables);

 

-            if (assignOperator != null) {

-                currentOperator = assignOperator;

-                parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

-            }

-        }

+			if (assignOperator != null) {

+				currentOperator = assignOperator;

+				parentOperator = new MutableObject<ILogicalOperator>(

+						currentOperator);

+			}

+		}

 

-        ILogicalOperator po = new ProjectOperator(variables);

-        po.getInputs().add(parentOperator);

-        t.rewriteOperatorOutputSchema(variables, operator);

-        return po;

-    }

+		ILogicalOperator po = new ProjectOperator(variables);

+		po.getInputs().add(parentOperator);

+		t.rewriteOperatorOutputSchema(variables, operator);

+		return po;

+	}

 

-    /**

-     * deal with reduce sink operator for the case of join

-     */

-    @Override

-    public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator, Mutable<ILogicalOperator> parentOperator,

-            Translator t) {

+	/**

+	 * deal with reduce sink operator for the case of join

+	 */

+	@Override

+	public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,

+			Mutable<ILogicalOperator> parentOperator, Translator t) {

 

-        Operator downStream = (Operator) operator.getChildOperators().get(0);

-        if (!(downStream instanceof JoinOperator))

-            return null;

+		Operator downStream = (Operator) operator.getChildOperators().get(0);

+		if (!(downStream instanceof JoinOperator))

+			return null;

 

-        ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

-        List<ExprNodeDesc> keys = desc.getKeyCols();

-        List<ExprNodeDesc> values = desc.getValueCols();

-        List<ExprNodeDesc> partitionCols = desc.getPartitionCols();

+		ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

+		List<ExprNodeDesc> keys = desc.getKeyCols();

+		List<ExprNodeDesc> values = desc.getValueCols();

+		List<ExprNodeDesc> partitionCols = desc.getPartitionCols();

 

-        /**

-         * rewrite key, value, paritioncol expressions

-         */

-        for (ExprNodeDesc key : keys)

-            t.rewriteExpression(key);

-        for (ExprNodeDesc value : values)

-            t.rewriteExpression(value);

-        for (ExprNodeDesc col : partitionCols)

-            t.rewriteExpression(col);

+		/**

+		 * rewrite key, value, paritioncol expressions

+		 */

+		for (ExprNodeDesc key : keys)

+			t.rewriteExpression(key);

+		for (ExprNodeDesc value : values)

+			t.rewriteExpression(value);

+		for (ExprNodeDesc col : partitionCols)

+			t.rewriteExpression(col);

 

-        ILogicalOperator currentOperator = null;

+		ILogicalOperator currentOperator = null;

 

-        // add assign operator for keys if necessary

-        ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

-        ILogicalOperator assignOperator = t.getAssignOperator(parentOperator, keys, keyVariables);

-        if (assignOperator != null) {

-            currentOperator = assignOperator;

-            parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

-        }

+		// add assign operator for keys if necessary

+		ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

+		ILogicalOperator assignOperator = t.getAssignOperator(parentOperator,

+				keys, keyVariables);

+		if (assignOperator != null) {

+			currentOperator = assignOperator;

+			parentOperator = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

 

-        // add assign operator for values if necessary

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

-        assignOperator = t.getAssignOperator(parentOperator, values, variables);

-        if (assignOperator != null) {

-            currentOperator = assignOperator;

-            parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

-        }

+		// add assign operator for values if necessary

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		assignOperator = t.getAssignOperator(parentOperator, values, variables);

+		if (assignOperator != null) {

+			currentOperator = assignOperator;

+			parentOperator = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

 

-        // unified schema: key, value

-        ArrayList<LogicalVariable> unifiedKeyValues = new ArrayList<LogicalVariable>();

-        unifiedKeyValues.addAll(keyVariables);

-        for (LogicalVariable value : variables)

-            if (keyVariables.indexOf(value) < 0)

-                unifiedKeyValues.add(value);

+		// unified schema: key, value

+		ArrayList<LogicalVariable> unifiedKeyValues = new ArrayList<LogicalVariable>();

+		unifiedKeyValues.addAll(keyVariables);

+		for (LogicalVariable value : variables)

+			if (keyVariables.indexOf(value) < 0)

+				unifiedKeyValues.add(value);

 

-        // insert projection operator, it is a *must*,

-        // in hive, reduce sink sometimes also do the projection operator's

-        // task

-        currentOperator = new ProjectOperator(unifiedKeyValues);

-        currentOperator.getInputs().add(parentOperator);

-        parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

+		// insert projection operator, it is a *must*,

+		// in hive, reduce sink sometimes also do the projection operator's

+		// task

+		currentOperator = new ProjectOperator(unifiedKeyValues);

+		currentOperator.getInputs().add(parentOperator);

+		parentOperator = new MutableObject<ILogicalOperator>(currentOperator);

 

-        reduceSinkToKeyVariables.put(operator, keyVariables);

-        List<String> fieldNames = new ArrayList<String>();

-        List<TypeInfo> types = new ArrayList<TypeInfo>();

-        for (LogicalVariable var : unifiedKeyValues) {

-            fieldNames.add(var.toString());

-            types.add(t.getType(var));

-        }

-        reduceSinkToFieldNames.put(operator, fieldNames);

-        reduceSinkToTypes.put(operator, types);

-        t.rewriteOperatorOutputSchema(variables, operator);

+		reduceSinkToKeyVariables.put(operator, keyVariables);

+		List<String> fieldNames = new ArrayList<String>();

+		List<TypeInfo> types = new ArrayList<TypeInfo>();

+		for (LogicalVariable var : unifiedKeyValues) {

+			fieldNames.add(var.toString());

+			types.add(t.getType(var));

+		}

+		reduceSinkToFieldNames.put(operator, fieldNames);

+		reduceSinkToTypes.put(operator, types);

+		t.rewriteOperatorOutputSchema(variables, operator);

 

-        latestAlgebricksOperator = currentOperator;

-        latestReduceSink = operator;

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		latestAlgebricksOperator = currentOperator;

+		latestReduceSink = operator;

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

-    /**

-     * partial rewrite a join operator

-     * 

-     * @param operator

-     * @param t

-     */

-    private void translateJoinOperatorPreprocess(Operator operator, Translator t) {

-        JoinDesc desc = (JoinDesc) operator.getConf();

-        ReduceSinkDesc reduceSinkDesc = (ReduceSinkDesc) latestReduceSink.getConf();

-        int tag = reduceSinkDesc.getTag();

+	/**

+	 * partial rewrite a join operator

+	 * 

+	 * @param operator

+	 * @param t

+	 */

+	private void translateJoinOperatorPreprocess(Operator operator, Translator t) {

+		JoinDesc desc = (JoinDesc) operator.getConf();

+		ReduceSinkDesc reduceSinkDesc = (ReduceSinkDesc) latestReduceSink

+				.getConf();

+		int tag = reduceSinkDesc.getTag();

 

-        Map<Byte, List<ExprNodeDesc>> exprMap = desc.getExprs();

-        List<ExprNodeDesc> exprs = exprMap.get(Byte.valueOf((byte) tag));

+		Map<Byte, List<ExprNodeDesc>> exprMap = desc.getExprs();

+		List<ExprNodeDesc> exprs = exprMap.get(Byte.valueOf((byte) tag));

 

-        for (ExprNodeDesc expr : exprs)

-            t.rewriteExpression(expr);

+		for (ExprNodeDesc expr : exprs)

+			t.rewriteExpression(expr);

 

-        List<Operator> parents = operatorToHiveParents.get(operator);

-        if (parents == null) {

-            parents = new ArrayList<Operator>();

-            operatorToHiveParents.put(operator, parents);

-        }

-        parents.add(latestReduceSink);

+		List<Operator> parents = operatorToHiveParents.get(operator);

+		if (parents == null) {

+			parents = new ArrayList<Operator>();

+			operatorToHiveParents.put(operator, parents);

+		}

+		parents.add(latestReduceSink);

 

-        List<ILogicalOperator> asterixParents = operatorToAsterixParents.get(operator);

-        if (asterixParents == null) {

-            asterixParents = new ArrayList<ILogicalOperator>();

-            operatorToAsterixParents.put(operator, asterixParents);

-        }

-        asterixParents.add(latestAlgebricksOperator);

-    }

+		List<ILogicalOperator> asterixParents = operatorToAsterixParents

+				.get(operator);

+		if (asterixParents == null) {

+			asterixParents = new ArrayList<ILogicalOperator>();

+			operatorToAsterixParents.put(operator, asterixParents);

+		}

+		asterixParents.add(latestAlgebricksOperator);

+	}

 

-    // generate a join tree from a list of exchange/reducesink operator

-    // both exchanges and reduce sinks have the same order

-    private ILogicalOperator generateJoinTree(List<JoinCondDesc> conds, List<ILogicalOperator> exchanges,

-            List<Operator> reduceSinks, int offset, Translator t) {

-        // get a list of reduce sink descs (input descs)

-        int inputSize = reduceSinks.size() - offset;

+	// generate a join tree from a list of exchange/reducesink operator

+	// both exchanges and reduce sinks have the same order

+	private ILogicalOperator generateJoinTree(List<JoinCondDesc> conds,

+			List<ILogicalOperator> exchanges, List<Operator> reduceSinks,

+			int offset, Translator t) {

+		// get a list of reduce sink descs (input descs)

+		int inputSize = reduceSinks.size() - offset;

 

-        if (inputSize == 2) {

-            ILogicalOperator currentRoot;

+		if (inputSize == 2) {

+			ILogicalOperator currentRoot;

 

-            List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();

-            for (int i = reduceSinks.size() - 1; i >= offset; i--)

-                reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i).getConf());

+			List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();

+			for (int i = reduceSinks.size() - 1; i >= offset; i--)

+				reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i)

+						.getConf());

 

-            // get the object inspector for the join

-            List<String> fieldNames = new ArrayList<String>();

-            List<TypeInfo> types = new ArrayList<TypeInfo>();

-            for (int i = reduceSinks.size() - 1; i >= offset; i--) {

-                fieldNames.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));

-                types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));

-            }

+			// get the object inspector for the join

+			List<String> fieldNames = new ArrayList<String>();

+			List<TypeInfo> types = new ArrayList<TypeInfo>();

+			for (int i = reduceSinks.size() - 1; i >= offset; i--) {

+				fieldNames

+						.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));

+				types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));

+			}

 

-            // get number of equality conjunctions in the final join condition

-            int size = reduceSinkDescs.get(0).getKeyCols().size();

+			// get number of equality conjunctions in the final join condition

+			int size = reduceSinkDescs.get(0).getKeyCols().size();

 

-            // make up the join conditon expression

-            List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

-                // create a join key pair

-                List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

-                for (ReduceSinkDesc sink : reduceSinkDescs) {

-                    keyPair.add(sink.getKeyCols().get(i));

-                }

-                // create a hive equal condition

-                ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,

-                        new GenericUDFOPEqual(), keyPair);

-                // add the equal condition to the conjunction list

-                joinConditionChildren.add(equality);

-            }

-            // get final conjunction expression

-            ExprNodeDesc conjunct = null;

+			// make up the join conditon expression

+			List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

+				// create a join key pair

+				List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

+				for (ReduceSinkDesc sink : reduceSinkDescs) {

+					keyPair.add(sink.getKeyCols().get(i));

+				}

+				// create a hive equal condition

+				ExprNodeDesc equality = new ExprNodeGenericFuncDesc(

+						TypeInfoFactory.booleanTypeInfo,

+						new GenericUDFOPEqual(), keyPair);

+				// add the equal condition to the conjunction list

+				joinConditionChildren.add(equality);

+			}

+			// get final conjunction expression

+			ExprNodeDesc conjunct = null;

 

-            if (joinConditionChildren.size() > 1)

-                conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

-                        joinConditionChildren);

-            else if (joinConditionChildren.size() == 1)

-                conjunct = joinConditionChildren.get(0);

-            else {

-                // there is no join equality condition, equal-join

-                conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));

-            }

-            // get an ILogicalExpression from hive's expression

-            Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);

+			if (joinConditionChildren.size() > 1)

+				conjunct = new ExprNodeGenericFuncDesc(

+						TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

+						joinConditionChildren);

+			else if (joinConditionChildren.size() == 1)

+				conjunct = joinConditionChildren.get(0);

+			else {

+				// there is no join equality condition, equal-join

+				conjunct = new ExprNodeConstantDesc(

+						TypeInfoFactory.booleanTypeInfo, new Boolean(true));

+			}

+			// get an ILogicalExpression from hive's expression

+			Mutable<ILogicalExpression> expression = t

+					.translateScalarFucntion(conjunct);

 

-            Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(exchanges.get(exchanges.size() - 1));

-            Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(exchanges.get(exchanges.size() - 2));

-            // get the join operator

-            if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {

-                currentRoot = new LeftOuterJoinOperator(expression);

-                Mutable<ILogicalOperator> temp = leftBranch;

-                leftBranch = rightBranch;

-                rightBranch = temp;

-            } else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {

-                currentRoot = new LeftOuterJoinOperator(expression);

-            } else

-                currentRoot = new InnerJoinOperator(expression);

+			Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(

+					exchanges.get(exchanges.size() - 1));

+			Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(

+					exchanges.get(exchanges.size() - 2));

+			// get the join operator

+			if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {

+				currentRoot = new LeftOuterJoinOperator(expression);

+				Mutable<ILogicalOperator> temp = leftBranch;

+				leftBranch = rightBranch;

+				rightBranch = temp;

+			} else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {

+				currentRoot = new LeftOuterJoinOperator(expression);

+			} else

+				currentRoot = new InnerJoinOperator(expression);

 

-            currentRoot.getInputs().add(leftBranch);

-            currentRoot.getInputs().add(rightBranch);

+			currentRoot.getInputs().add(leftBranch);

+			currentRoot.getInputs().add(rightBranch);

 

-            // rewriteOperatorOutputSchema(variables, operator);

-            return currentRoot;

-        } else {

-            // get the child join operator and insert and one-to-one exchange

-            ILogicalOperator joinSrcOne = generateJoinTree(conds, exchanges, reduceSinks, offset + 1, t);

-            // joinSrcOne.addInput(childJoin);

+			// rewriteOperatorOutputSchema(variables, operator);

+			return currentRoot;

+		} else {

+			// get the child join operator and insert and one-to-one exchange

+			ILogicalOperator joinSrcOne = generateJoinTree(conds, exchanges,

+					reduceSinks, offset + 1, t);

+			// joinSrcOne.addInput(childJoin);

 

-            ILogicalOperator currentRoot;

+			ILogicalOperator currentRoot;

 

-            List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();

-            for (int i = offset; i < offset + 2; i++)

-                reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i).getConf());

+			List<ReduceSinkDesc> reduceSinkDescs = new ArrayList<ReduceSinkDesc>();

+			for (int i = offset; i < offset + 2; i++)

+				reduceSinkDescs.add((ReduceSinkDesc) reduceSinks.get(i)

+						.getConf());

 

-            // get the object inspector for the join

-            List<String> fieldNames = new ArrayList<String>();

-            List<TypeInfo> types = new ArrayList<TypeInfo>();

-            for (int i = offset; i < reduceSinks.size(); i++) {

-                fieldNames.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));

-                types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));

-            }

+			// get the object inspector for the join

+			List<String> fieldNames = new ArrayList<String>();

+			List<TypeInfo> types = new ArrayList<TypeInfo>();

+			for (int i = offset; i < reduceSinks.size(); i++) {

+				fieldNames

+						.addAll(reduceSinkToFieldNames.get(reduceSinks.get(i)));

+				types.addAll(reduceSinkToTypes.get(reduceSinks.get(i)));

+			}

 

-            // get number of equality conjunctions in the final join condition

-            int size = reduceSinkDescs.get(0).getKeyCols().size();

+			// get number of equality conjunctions in the final join condition

+			int size = reduceSinkDescs.get(0).getKeyCols().size();

 

-            // make up the join condition expression

-            List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

-                // create a join key pair

-                List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

-                for (ReduceSinkDesc sink : reduceSinkDescs) {

-                    keyPair.add(sink.getKeyCols().get(i));

-                }

-                // create a hive equal condition

-                ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,

-                        new GenericUDFOPEqual(), keyPair);

-                // add the equal condition to the conjunction list

-                joinConditionChildren.add(equality);

-            }

-            // get final conjunction expression

-            ExprNodeDesc conjunct = null;

+			// make up the join condition expression

+			List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

+				// create a join key pair

+				List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

+				for (ReduceSinkDesc sink : reduceSinkDescs) {

+					keyPair.add(sink.getKeyCols().get(i));

+				}

+				// create a hive equal condition

+				ExprNodeDesc equality = new ExprNodeGenericFuncDesc(

+						TypeInfoFactory.booleanTypeInfo,

+						new GenericUDFOPEqual(), keyPair);

+				// add the equal condition to the conjunction list

+				joinConditionChildren.add(equality);

+			}

+			// get final conjunction expression

+			ExprNodeDesc conjunct = null;

 

-            if (joinConditionChildren.size() > 1)

-                conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

-                        joinConditionChildren);

-            else if (joinConditionChildren.size() == 1)

-                conjunct = joinConditionChildren.get(0);

-            else {

-                // there is no join equality condition, full outer join

-                conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));

-            }

-            // get an ILogicalExpression from hive's expression

-            Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);

+			if (joinConditionChildren.size() > 1)

+				conjunct = new ExprNodeGenericFuncDesc(

+						TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

+						joinConditionChildren);

+			else if (joinConditionChildren.size() == 1)

+				conjunct = joinConditionChildren.get(0);

+			else {

+				// there is no join equality condition, full outer join

+				conjunct = new ExprNodeConstantDesc(

+						TypeInfoFactory.booleanTypeInfo, new Boolean(true));

+			}

+			// get an ILogicalExpression from hive's expression

+			Mutable<ILogicalExpression> expression = t

+					.translateScalarFucntion(conjunct);

 

-            Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(joinSrcOne);

-            Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(exchanges.get(offset));

+			Mutable<ILogicalOperator> leftBranch = new MutableObject<ILogicalOperator>(

+					joinSrcOne);

+			Mutable<ILogicalOperator> rightBranch = new MutableObject<ILogicalOperator>(

+					exchanges.get(offset));

 

-            // get the join operator

-            if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {

-                currentRoot = new LeftOuterJoinOperator(expression);

-                Mutable<ILogicalOperator> temp = leftBranch;

-                leftBranch = rightBranch;

-                rightBranch = temp;

-            } else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {

-                currentRoot = new LeftOuterJoinOperator(expression);

-            } else

-                currentRoot = new InnerJoinOperator(expression);

+			// get the join operator

+			if (conds.get(offset).getType() == JoinDesc.LEFT_OUTER_JOIN) {

+				currentRoot = new LeftOuterJoinOperator(expression);

+				Mutable<ILogicalOperator> temp = leftBranch;

+				leftBranch = rightBranch;

+				rightBranch = temp;

+			} else if (conds.get(offset).getType() == JoinDesc.RIGHT_OUTER_JOIN) {

+				currentRoot = new LeftOuterJoinOperator(expression);

+			} else

+				currentRoot = new InnerJoinOperator(expression);

 

-            // set the inputs from Algebricks join operator

-            // add the current table

-            currentRoot.getInputs().add(leftBranch);

-            currentRoot.getInputs().add(rightBranch);

+			// set the inputs from Algebricks join operator

+			// add the current table

+			currentRoot.getInputs().add(leftBranch);

+			currentRoot.getInputs().add(rightBranch);

 

-            return currentRoot;

-        }

-    }

+			return currentRoot;

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
index 56b0ef4..004a8c2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LateralViewJoinVisitor.java
@@ -53,69 +53,72 @@
  * previous LVJ operator.

  */

 

+@SuppressWarnings("rawtypes")

 public class LateralViewJoinVisitor extends DefaultVisitor {

 

-    private UDTFDesc udtf;

+	private UDTFDesc udtf;

 

-    private List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();

+	private List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();

 

-    @Override

-    public Mutable<ILogicalOperator> visit(LateralViewJoinOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {

+	@Override

+	public Mutable<ILogicalOperator> visit(LateralViewJoinOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException {

 

-        parents.add(AlgebricksParentOperatorRef);

-        if (operator.getParentOperators().size() > parents.size()) {

-            return null;

-        }

+		parents.add(AlgebricksParentOperatorRef);

+		if (operator.getParentOperators().size() > parents.size()) {

+			return null;

+		}

 

-        Operator parent0 = operator.getParentOperators().get(0);

-        Operator parent1 = operator.getParentOperators().get(1);

-        List<LogicalVariable> variables;

+		Operator parent0 = operator.getParentOperators().get(0);

+		ILogicalOperator parentOperator;

+		ILogicalExpression unnestArg;

+		if (parent0 instanceof UDTFOperator) {

+			List<LogicalVariable> unnestVars = new ArrayList<LogicalVariable>();

+			VariableUtilities.getLiveVariables(parents.get(1).getValue(),

+					unnestVars);

+			unnestArg = new VariableReferenceExpression(unnestVars.get(0));

+			parentOperator = parents.get(1).getValue();

+		} else {

+			List<LogicalVariable> unnestVars = new ArrayList<LogicalVariable>();

+			VariableUtilities.getLiveVariables(parents.get(0).getValue(),

+					unnestVars);

+			unnestArg = new VariableReferenceExpression(unnestVars.get(0));

+			parentOperator = parents.get(0).getValue();

+		}

 

-        ILogicalOperator parentOperator;

-        ILogicalExpression unnestArg;

-        if (parent0 instanceof UDTFOperator) {

-            variables = t.getVariablesFromSchema(t.generateInputSchema(parent1));

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

-            VariableUtilities.getLiveVariables(parents.get(1).getValue(), unnestVars);

-            unnestArg = new VariableReferenceExpression(unnestVars.get(0));

-            parentOperator = parents.get(1).getValue();

-        } else {

-            variables = t.getVariablesFromSchema(t.generateInputSchema(parent0));

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

-            VariableUtilities.getLiveVariables(parents.get(0).getValue(), unnestVars);

-            unnestArg = new VariableReferenceExpression(unnestVars.get(0));

-            parentOperator = parents.get(0).getValue();

-        }

+		LogicalVariable var = t.getVariable(udtf.toString(),

+				TypeInfoFactory.unknownTypeInfo);

 

-        LogicalVariable var = t.getVariable(udtf.toString(), TypeInfoFactory.unknownTypeInfo);

+		Mutable<ILogicalExpression> unnestExpr = t.translateUnnestFunction(

+				udtf, new MutableObject<ILogicalExpression>(unnestArg));

+		ILogicalOperator currentOperator = new UnnestOperator(var, unnestExpr);

 

-        Mutable<ILogicalExpression> unnestExpr = t.translateUnnestFunction(udtf, new MutableObject<ILogicalExpression>(

-                unnestArg));

-        ILogicalOperator currentOperator = new UnnestOperator(var, unnestExpr);

+		List<LogicalVariable> outputVars = new ArrayList<LogicalVariable>();

+		VariableUtilities.getLiveVariables(parentOperator, outputVars);

+		outputVars.add(var);

+		currentOperator.getInputs().add(

+				new MutableObject<ILogicalOperator>(parentOperator));

 

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

-        VariableUtilities.getLiveVariables(parentOperator, outputVars);

-        outputVars.add(var);

-        currentOperator.getInputs().add(new MutableObject<ILogicalOperator>(parentOperator));

+		parents.clear();

+		udtf = null;

+		t.rewriteOperatorOutputSchema(outputVars, operator);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

-        parents.clear();

-        udtf = null;

-        t.rewriteOperatorOutputSchema(outputVars, operator);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(UDTFOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		Schema currentSchema = t.generateInputSchema(operator

+				.getParentOperators().get(0));

+		udtf = (UDTFDesc) operator.getConf();

 

-    @Override

-    public Mutable<ILogicalOperator> visit(UDTFOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

-        Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));

-        udtf = (UDTFDesc) operator.getConf();

-

-        // populate the schema from upstream operator

-        operator.setSchema(operator.getParentOperators().get(0).getSchema());

-        List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);

-        t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

-        return null;

-    }

+		// populate the schema from upstream operator

+		operator.setSchema(operator.getParentOperators().get(0).getSchema());

+		List<LogicalVariable> latestOutputSchema = t

+				.getVariablesFromSchema(currentSchema);

+		t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java
index d7d6d67..84cdf00 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/LimitVisitor.java
@@ -18,24 +18,27 @@
 

 public class LimitVisitor extends DefaultVisitor {

 

-    @Override

-    public Mutable<ILogicalOperator> visit(LimitOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) {

-        Schema currentSchema = t.generateInputSchema(operator.getParentOperators().get(0));

+	@Override

+	public Mutable<ILogicalOperator> visit(LimitOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		Schema currentSchema = t.generateInputSchema(operator

+				.getParentOperators().get(0));

 

-        LimitDesc desc = (LimitDesc) operator.getConf();

-        int limit = desc.getLimit();

-        Integer limitValue = new Integer(limit);

+		LimitDesc desc = (LimitDesc) operator.getConf();

+		int limit = desc.getLimit();

+		Integer limitValue = new Integer(limit);

 

-        ILogicalExpression expr = new ConstantExpression(new HivesterixConstantValue(limitValue));

-        ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator(

-                expr, true);

-        currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+		ILogicalExpression expr = new ConstantExpression(

+				new HivesterixConstantValue(limitValue));

+		ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator(

+				expr, true);

+		currentOperator.getInputs().add(AlgebricksParentOperatorRef);

 

-        operator.setSchema(operator.getParentOperators().get(0).getSchema());

-        List<LogicalVariable> latestOutputSchema = t.getVariablesFromSchema(currentSchema);

-        t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		operator.setSchema(operator.getParentOperators().get(0).getSchema());

+		List<LogicalVariable> latestOutputSchema = t

+				.getVariablesFromSchema(currentSchema);

+		t.rewriteOperatorOutputSchema(latestOutputSchema, operator);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
index f8af063..fa5d014 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/MapJoinVisitor.java
@@ -31,141 +31,153 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

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

 

+@SuppressWarnings("rawtypes")

 public class MapJoinVisitor extends DefaultVisitor {

 

-    /**

-     * map a join operator (in hive) to its parent operators (in asterix)

-     */

-    private HashMap<Operator, List<Mutable<ILogicalOperator>>> opMap = new HashMap<Operator, List<Mutable<ILogicalOperator>>>();

+	/**

+	 * map a join operator (in hive) to its parent operators (in asterix)

+	 */

+	private HashMap<Operator, List<Mutable<ILogicalOperator>>> opMap = new HashMap<Operator, List<Mutable<ILogicalOperator>>>();

 

-    @Override

-    public Mutable<ILogicalOperator> visit(MapJoinOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

-        List<Operator<? extends Serializable>> joinSrc = operator.getParentOperators();

-        List<Mutable<ILogicalOperator>> parents = opMap.get(operator);

-        if (parents == null) {

-            parents = new ArrayList<Mutable<ILogicalOperator>>();

-            opMap.put(operator, parents);

-        }

-        parents.add(AlgebricksParentOperatorRef);

-        if (joinSrc.size() != parents.size())

-            return null;

+	@Override

+	public Mutable<ILogicalOperator> visit(MapJoinOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) {

+		List<Operator<? extends Serializable>> joinSrc = operator

+				.getParentOperators();

+		List<Mutable<ILogicalOperator>> parents = opMap.get(operator);

+		if (parents == null) {

+			parents = new ArrayList<Mutable<ILogicalOperator>>();

+			opMap.put(operator, parents);

+		}

+		parents.add(AlgebricksParentOperatorRef);

+		if (joinSrc.size() != parents.size())

+			return null;

 

-        ILogicalOperator currentOperator;

-        // make an map join operator

-        // TODO: will have trouble for n-way joins

-        MapJoinDesc joinDesc = (MapJoinDesc) operator.getConf();

+		ILogicalOperator currentOperator;

+		// make an map join operator

+		// TODO: will have trouble for n-way joins

+		MapJoinDesc joinDesc = (MapJoinDesc) operator.getConf();

 

-        Map<Byte, List<ExprNodeDesc>> keyMap = joinDesc.getKeys();

-        // get the projection expression (already re-written) from each source

-        // table

-        Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();

+		Map<Byte, List<ExprNodeDesc>> keyMap = joinDesc.getKeys();

+		// get the projection expression (already re-written) from each source

+		// table

+		Map<Byte, List<ExprNodeDesc>> exprMap = joinDesc.getExprs();

 

-        int inputSize = operator.getParentOperators().size();

-        // get a list of reduce sink descs (input descs)

+		int inputSize = operator.getParentOperators().size();

+		// get a list of reduce sink descs (input descs)

 

-        // get the parent operator

-        List<Mutable<ILogicalOperator>> parentOps = parents;

+		// get the parent operator

+		List<Mutable<ILogicalOperator>> parentOps = parents;

 

-        List<String> fieldNames = new ArrayList<String>();

-        List<TypeInfo> types = new ArrayList<TypeInfo>();

-        for (Operator ts : joinSrc) {

-            List<ColumnInfo> columns = ts.getSchema().getSignature();

-            for (ColumnInfo col : columns) {

-                fieldNames.add(col.getInternalName());

-                types.add(col.getType());

-            }

-        }

+		List<String> fieldNames = new ArrayList<String>();

+		List<TypeInfo> types = new ArrayList<TypeInfo>();

+		for (Operator ts : joinSrc) {

+			List<ColumnInfo> columns = ts.getSchema().getSignature();

+			for (ColumnInfo col : columns) {

+				fieldNames.add(col.getInternalName());

+				types.add(col.getType());

+			}

+		}

 

-        // get number of equality conjunctions in the final join condition

-        Set<Entry<Byte, List<ExprNodeDesc>>> keyEntries = keyMap.entrySet();

-        List<List<ExprNodeDesc>> keyLists = new ArrayList<List<ExprNodeDesc>>();

-        Iterator<Entry<Byte, List<ExprNodeDesc>>> entry = keyEntries.iterator();

+		// get number of equality conjunctions in the final join condition

+		Set<Entry<Byte, List<ExprNodeDesc>>> keyEntries = keyMap.entrySet();

+		Iterator<Entry<Byte, List<ExprNodeDesc>>> entry = keyEntries.iterator();

 

-        int size = 0;

-        if (entry.hasNext())

-            size = entry.next().getValue().size();

+		int size = 0;

+		if (entry.hasNext())

+			size = entry.next().getValue().size();

 

-        // make up the join conditon expression

-        List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

-            // create a join key pair

-            List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

-            for (int j = 0; j < inputSize; j++) {

-                keyPair.add(keyMap.get(Byte.valueOf((byte) j)).get(i));

-            }

-            // create a hive equal condition

-            ExprNodeDesc equality = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,

-                    new GenericUDFOPEqual(), keyPair);

-            // add the equal condition to the conjunction list

-            joinConditionChildren.add(equality);

-        }

-        // get final conjunction expression

-        ExprNodeDesc conjunct = null;

+		// make up the join conditon expression

+		List<ExprNodeDesc> joinConditionChildren = new ArrayList<ExprNodeDesc>();

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

+			// create a join key pair

+			List<ExprNodeDesc> keyPair = new ArrayList<ExprNodeDesc>();

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

+				keyPair.add(keyMap.get(Byte.valueOf((byte) j)).get(i));

+			}

+			// create a hive equal condition

+			ExprNodeDesc equality = new ExprNodeGenericFuncDesc(

+					TypeInfoFactory.booleanTypeInfo, new GenericUDFOPEqual(),

+					keyPair);

+			// add the equal condition to the conjunction list

+			joinConditionChildren.add(equality);

+		}

+		// get final conjunction expression

+		ExprNodeDesc conjunct = null;

 

-        if (joinConditionChildren.size() > 1)

-            conjunct = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

-                    joinConditionChildren);

-        else if (joinConditionChildren.size() == 1)

-            conjunct = joinConditionChildren.get(0);

-        else {

-            // there is no join equality condition, full outer join

-            conjunct = new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, new Boolean(true));

-        }

-        // get an ILogicalExpression from hive's expression

-        Mutable<ILogicalExpression> expression = t.translateScalarFucntion(conjunct);

+		if (joinConditionChildren.size() > 1)

+			conjunct = new ExprNodeGenericFuncDesc(

+					TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(),

+					joinConditionChildren);

+		else if (joinConditionChildren.size() == 1)

+			conjunct = joinConditionChildren.get(0);

+		else {

+			// there is no join equality condition, full outer join

+			conjunct = new ExprNodeConstantDesc(

+					TypeInfoFactory.booleanTypeInfo, new Boolean(true));

+		}

+		// get an ILogicalExpression from hive's expression

+		Mutable<ILogicalExpression> expression = t

+				.translateScalarFucntion(conjunct);

 

-        ArrayList<LogicalVariable> left = new ArrayList<LogicalVariable>();

-        ArrayList<LogicalVariable> right = new ArrayList<LogicalVariable>();

+		ArrayList<LogicalVariable> left = new ArrayList<LogicalVariable>();

+		ArrayList<LogicalVariable> right = new ArrayList<LogicalVariable>();

 

-        Set<Entry<Byte, List<ExprNodeDesc>>> kentries = keyMap.entrySet();

-        Iterator<Entry<Byte, List<ExprNodeDesc>>> kiterator = kentries.iterator();

-        int iteration = 0;

-        ILogicalOperator assignOperator = null;

-        while (kiterator.hasNext()) {

-            List<ExprNodeDesc> outputExprs = kiterator.next().getValue();

+		Set<Entry<Byte, List<ExprNodeDesc>>> kentries = keyMap.entrySet();

+		Iterator<Entry<Byte, List<ExprNodeDesc>>> kiterator = kentries

+				.iterator();

+		int iteration = 0;

+		ILogicalOperator assignOperator = null;

+		while (kiterator.hasNext()) {

+			List<ExprNodeDesc> outputExprs = kiterator.next().getValue();

 

-            if (iteration == 0)

-                assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, left);

-            else

-                assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, right);

+			if (iteration == 0)

+				assignOperator = t.getAssignOperator(

+						AlgebricksParentOperatorRef, outputExprs, left);

+			else

+				assignOperator = t.getAssignOperator(

+						AlgebricksParentOperatorRef, outputExprs, right);

 

-            if (assignOperator != null) {

-                currentOperator = assignOperator;

-                AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-            }

-            iteration++;

-        }

+			if (assignOperator != null) {

+				currentOperator = assignOperator;

+				AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+						currentOperator);

+			}

+			iteration++;

+		}

 

-        List<Mutable<ILogicalOperator>> inputs = parentOps;

+		List<Mutable<ILogicalOperator>> inputs = parentOps;

 

-        // get the join operator

-        currentOperator = new InnerJoinOperator(expression);

+		// get the join operator

+		currentOperator = new InnerJoinOperator(expression);

 

-        // set the inputs from asterix join operator

-        for (Mutable<ILogicalOperator> input : inputs)

-            currentOperator.getInputs().add(input);

-        AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

+		// set the inputs from asterix join operator

+		for (Mutable<ILogicalOperator> input : inputs)

+			currentOperator.getInputs().add(input);

+		AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+				currentOperator);

 

-        // add assign and project operator

-        // output variables

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

-        Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();

-        Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();

-        while (iterator.hasNext()) {

-            List<ExprNodeDesc> outputExprs = iterator.next().getValue();

-            assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, outputExprs, variables);

+		// add assign and project operator

+		// output variables

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		Set<Entry<Byte, List<ExprNodeDesc>>> entries = exprMap.entrySet();

+		Iterator<Entry<Byte, List<ExprNodeDesc>>> iterator = entries.iterator();

+		while (iterator.hasNext()) {

+			List<ExprNodeDesc> outputExprs = iterator.next().getValue();

+			assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef,

+					outputExprs, variables);

 

-            if (assignOperator != null) {

-                currentOperator = assignOperator;

-                AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-            }

-        }

+			if (assignOperator != null) {

+				currentOperator = assignOperator;

+				AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+						currentOperator);

+			}

+		}

 

-        currentOperator = new ProjectOperator(variables);

-        currentOperator.getInputs().add(AlgebricksParentOperatorRef);

-        t.rewriteOperatorOutputSchema(variables, operator);

-        // opMap.clear();

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		currentOperator = new ProjectOperator(variables);

+		currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+		t.rewriteOperatorOutputSchema(variables, operator);

+		// opMap.clear();

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java
index eb0922f..0d2067c 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/ProjectVisitor.java
@@ -17,40 +17,42 @@
 

 public class ProjectVisitor extends DefaultVisitor {

 

-    /**

-     * translate project operator

-     */

-    @Override

-    public Mutable<ILogicalOperator> visit(SelectOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) {

+	/**

+	 * translate project operator

+	 */

+	@Override

+	public Mutable<ILogicalOperator> visit(SelectOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) {

 

-        SelectDesc desc = (SelectDesc) operator.getConf();

+		SelectDesc desc = (SelectDesc) operator.getConf();

 

-        if (desc == null)

-            return null;

+		if (desc == null)

+			return null;

 

-        List<ExprNodeDesc> cols = desc.getColList();

+		List<ExprNodeDesc> cols = desc.getColList();

 

-        if (cols == null)

-            return null;

+		if (cols == null)

+			return null;

 

-        // insert assign operator if necessary

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		// insert assign operator if necessary

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

 

-        for (ExprNodeDesc expr : cols)

-            t.rewriteExpression(expr);

+		for (ExprNodeDesc expr : cols)

+			t.rewriteExpression(expr);

 

-        ILogicalOperator assignOp = t.getAssignOperator(AlgebricksParentOperator, cols, variables);

-        ILogicalOperator currentOperator = null;

-        if (assignOp != null) {

-            currentOperator = assignOp;

-            AlgebricksParentOperator = new MutableObject<ILogicalOperator>(currentOperator);

-        }

+		ILogicalOperator assignOp = t.getAssignOperator(

+				AlgebricksParentOperator, cols, variables);

+		ILogicalOperator currentOperator = null;

+		if (assignOp != null) {

+			currentOperator = assignOp;

+			AlgebricksParentOperator = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

 

-        currentOperator = new ProjectOperator(variables);

-        currentOperator.getInputs().add(AlgebricksParentOperator);

-        t.rewriteOperatorOutputSchema(variables, operator);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		currentOperator = new ProjectOperator(variables);

+		currentOperator.getInputs().add(AlgebricksParentOperator);

+		t.rewriteOperatorOutputSchema(variables, operator);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java
index a4fc765..a2c0d03 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/SortVisitor.java
@@ -26,87 +26,100 @@
 

 public class SortVisitor extends DefaultVisitor {

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException {

-        ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

-        Operator downStream = (Operator) operator.getChildOperators().get(0);

-        List<ExprNodeDesc> keys = desc.getKeyCols();

-        if (!(downStream instanceof ExtractOperator && desc.getNumReducers() == 1 && keys.size() > 0)) {

-            return null;

-        }

+	@SuppressWarnings("rawtypes")

+	@Override

+	public Mutable<ILogicalOperator> visit(ReduceSinkOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException {

+		ReduceSinkDesc desc = (ReduceSinkDesc) operator.getConf();

+		Operator downStream = (Operator) operator.getChildOperators().get(0);

+		List<ExprNodeDesc> keys = desc.getKeyCols();

+		if (!(downStream instanceof ExtractOperator

+				&& desc.getNumReducers() == 1 && keys.size() > 0)) {

+			return null;

+		}

 

-        List<ExprNodeDesc> schema = new ArrayList<ExprNodeDesc>();

-        List<ExprNodeDesc> values = desc.getValueCols();

-        List<ExprNodeDesc> partitionCols = desc.getPartitionCols();

-        for (ExprNodeDesc key : keys) {

-            t.rewriteExpression(key);

-        }

-        for (ExprNodeDesc value : values) {

-            t.rewriteExpression(value);

-        }

-        for (ExprNodeDesc col : partitionCols) {

-            t.rewriteExpression(col);

-        }

+		List<ExprNodeDesc> schema = new ArrayList<ExprNodeDesc>();

+		List<ExprNodeDesc> values = desc.getValueCols();

+		List<ExprNodeDesc> partitionCols = desc.getPartitionCols();

+		for (ExprNodeDesc key : keys) {

+			t.rewriteExpression(key);

+		}

+		for (ExprNodeDesc value : values) {

+			t.rewriteExpression(value);

+		}

+		for (ExprNodeDesc col : partitionCols) {

+			t.rewriteExpression(col);

+		}

 

-        // add a order-by operator and limit if any

-        List<Pair<IOrder, Mutable<ILogicalExpression>>> pairs = new ArrayList<Pair<IOrder, Mutable<ILogicalExpression>>>();

-        char[] orders = desc.getOrder().toCharArray();

-        int i = 0;

-        for (ExprNodeDesc key : keys) {

-            Mutable<ILogicalExpression> expr = t.translateScalarFucntion(key);

-            IOrder order = orders[i] == '+' ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;

+		// add a order-by operator and limit if any

+		List<Pair<IOrder, Mutable<ILogicalExpression>>> pairs = new ArrayList<Pair<IOrder, Mutable<ILogicalExpression>>>();

+		char[] orders = desc.getOrder().toCharArray();

+		int i = 0;

+		for (ExprNodeDesc key : keys) {

+			Mutable<ILogicalExpression> expr = t.translateScalarFucntion(key);

+			IOrder order = orders[i] == '+' ? OrderOperator.ASC_ORDER

+					: OrderOperator.DESC_ORDER;

 

-            Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<IOrder, Mutable<ILogicalExpression>>(order, expr);

-            pairs.add(pair);

-            i++;

-        }

+			Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<IOrder, Mutable<ILogicalExpression>>(

+					order, expr);

+			pairs.add(pair);

+			i++;

+		}

 

-        // get input variables

-        ArrayList<LogicalVariable> inputVariables = new ArrayList<LogicalVariable>();

-        VariableUtilities.getProducedVariables(AlgebricksParentOperatorRef.getValue(), inputVariables);

+		// get input variables

+		ArrayList<LogicalVariable> inputVariables = new ArrayList<LogicalVariable>();

+		VariableUtilities.getProducedVariables(

+				AlgebricksParentOperatorRef.getValue(), inputVariables);

 

-        ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

-        ILogicalOperator currentOperator;

-        ILogicalOperator assignOp = t.getAssignOperator(AlgebricksParentOperatorRef, keys, keyVariables);

-        if (assignOp != null) {

-            currentOperator = assignOp;

-            AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-        }

+		ArrayList<LogicalVariable> keyVariables = new ArrayList<LogicalVariable>();

+		ILogicalOperator currentOperator;

+		ILogicalOperator assignOp = t.getAssignOperator(

+				AlgebricksParentOperatorRef, keys, keyVariables);

+		if (assignOp != null) {

+			currentOperator = assignOp;

+			AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

 

-        OrderColumn[] keyColumns = new OrderColumn[keyVariables.size()];

+		OrderColumn[] keyColumns = new OrderColumn[keyVariables.size()];

 

-        for (int j = 0; j < keyColumns.length; j++)

-            keyColumns[j] = new OrderColumn(keyVariables.get(j), pairs.get(j).first.getKind());

+		for (int j = 0; j < keyColumns.length; j++)

+			keyColumns[j] = new OrderColumn(keyVariables.get(j),

+					pairs.get(j).first.getKind());

 

-        // handle order operator

-        currentOperator = new OrderOperator(pairs);

-        currentOperator.getInputs().add(AlgebricksParentOperatorRef);

-        AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

+		// handle order operator

+		currentOperator = new OrderOperator(pairs);

+		currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+		AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+				currentOperator);

 

-        // project back, remove generated sort-key columns if any

-        if (assignOp != null) {

-            currentOperator = new ProjectOperator(inputVariables);

-            currentOperator.getInputs().add(AlgebricksParentOperatorRef);

-            AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-        }

+		// project back, remove generated sort-key columns if any

+		if (assignOp != null) {

+			currentOperator = new ProjectOperator(inputVariables);

+			currentOperator.getInputs().add(AlgebricksParentOperatorRef);

+			AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

 

-        /**

-         * a special rule for hive's order by output schema of reduce sink

-         * operator only contains the columns

-         */

-        for (ExprNodeDesc value : values) {

-            schema.add(value);

-        }

+		/**

+		 * a special rule for hive's order by output schema of reduce sink

+		 * operator only contains the columns

+		 */

+		for (ExprNodeDesc value : values) {

+			schema.add(value);

+		}

 

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

-        ILogicalOperator assignOperator = t.getAssignOperator(AlgebricksParentOperatorRef, schema, variables);

-        t.rewriteOperatorOutputSchema(variables, operator);

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		ILogicalOperator assignOperator = t.getAssignOperator(

+				AlgebricksParentOperatorRef, schema, variables);

+		t.rewriteOperatorOutputSchema(variables, operator);

 

-        if (assignOperator != null) {

-            currentOperator = assignOperator;

-            AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(currentOperator);

-        }

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		if (assignOperator != null) {

+			currentOperator = assignOperator;

+			AlgebricksParentOperatorRef = new MutableObject<ILogicalOperator>(

+					currentOperator);

+		}

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
index 6071716..3e12bb9 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/TableScanWriteVisitor.java
@@ -34,103 +34,115 @@
 

 public class TableScanWriteVisitor extends DefaultVisitor {

 

-    /**

-     * map from alias to partition desc

-     */

-    private HashMap<String, PartitionDesc> aliasToPathMap;

+	/**

+	 * map from alias to partition desc

+	 */

+	private HashMap<String, PartitionDesc> aliasToPathMap;

 

-    /**

-     * map from partition desc to data source

-     */

-    private HashMap<PartitionDesc, IDataSource<PartitionDesc>> dataSourceMap = new HashMap<PartitionDesc, IDataSource<PartitionDesc>>();

+	/**

+	 * map from partition desc to data source

+	 */

+	private HashMap<PartitionDesc, IDataSource<PartitionDesc>> dataSourceMap = new HashMap<PartitionDesc, IDataSource<PartitionDesc>>();

 

-    /**

-     * constructor

-     * 

-     * @param aliasToPathMap

-     */

-    public TableScanWriteVisitor(HashMap<String, PartitionDesc> aliasToPathMap) {

-        this.aliasToPathMap = aliasToPathMap;

-    }

+	/**

+	 * constructor

+	 * 

+	 * @param aliasToPathMap

+	 */

+	public TableScanWriteVisitor(HashMap<String, PartitionDesc> aliasToPathMap) {

+		this.aliasToPathMap = aliasToPathMap;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(TableScanOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        TableScanDesc desc = (TableScanDesc) operator.getConf();

-        if (desc == null) {

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

-            VariableUtilities.getLiveVariables(AlgebricksParentOperator.getValue(), schema);

-            t.rewriteOperatorOutputSchema(schema, operator);

-            return null;

-        }

+	@Override

+	public Mutable<ILogicalOperator> visit(TableScanOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		TableScanDesc desc = (TableScanDesc) operator.getConf();

+		if (desc == null) {

+			List<LogicalVariable> schema = new ArrayList<LogicalVariable>();

+			VariableUtilities.getLiveVariables(

+					AlgebricksParentOperator.getValue(), schema);

+			t.rewriteOperatorOutputSchema(schema, operator);

+			return null;

+		}

 

-        List<ColumnInfo> columns = operator.getSchema().getSignature();

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

-            if (columns.get(i).getIsVirtualCol() == true)

-                columns.remove(i);

+		List<ColumnInfo> columns = operator.getSchema().getSignature();

+		for (int i = columns.size() - 1; i >= 0; i--)

+			if (columns.get(i).getIsVirtualCol() == true)

+				columns.remove(i);

 

-        // start with empty tuple operator

-        List<TypeInfo> types = new ArrayList<TypeInfo>();

-        ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

-        List<String> names = new ArrayList<String>();

-        for (ColumnInfo column : columns) {

-            types.add(column.getType());

+		// start with empty tuple operator

+		List<TypeInfo> types = new ArrayList<TypeInfo>();

+		ArrayList<LogicalVariable> variables = new ArrayList<LogicalVariable>();

+		List<String> names = new ArrayList<String>();

+		for (ColumnInfo column : columns) {

+			types.add(column.getType());

 

-            LogicalVariable var = t.getVariableFromFieldName(column.getTabAlias() + "." + column.getInternalName());

-            LogicalVariable varNew;

+			LogicalVariable var = t.getVariableFromFieldName(column

+					.getTabAlias() + "." + column.getInternalName());

+			LogicalVariable varNew;

 

-            if (var != null) {

-                varNew = t.getVariable(

-                        column.getTabAlias() + "." + column.getInternalName() + operator.toString(),

-                        column.getType());

-                t.replaceVariable(var, varNew);

-                var = varNew;

-            } else

-                var = t.getNewVariable(column.getTabAlias() + "." + column.getInternalName(), column.getType());

+			if (var != null) {

+				varNew = t.getVariable(

+						column.getTabAlias() + "." + column.getInternalName()

+								+ operator.toString(), column.getType());

+				t.replaceVariable(var, varNew);

+				var = varNew;

+			} else

+				var = t.getNewVariable(

+						column.getTabAlias() + "." + column.getInternalName(),

+						column.getType());

 

-            variables.add(var);

-            names.add(column.getInternalName());

-        }

-        Schema currentSchema = new Schema(names, types);

+			variables.add(var);

+			names.add(column.getInternalName());

+		}

+		Schema currentSchema = new Schema(names, types);

 

-        String alias = desc.getAlias();

-        PartitionDesc partDesc = aliasToPathMap.get(alias);

-        IDataSource<PartitionDesc> dataSource = new HiveDataSource<PartitionDesc>(partDesc, currentSchema.getSchema());

-        ILogicalOperator currentOperator = new DataSourceScanOperator(variables, dataSource);

+		String alias = desc.getAlias();

+		PartitionDesc partDesc = aliasToPathMap.get(alias);

+		IDataSource<PartitionDesc> dataSource = new HiveDataSource<PartitionDesc>(

+				partDesc, currentSchema.getSchema());

+		ILogicalOperator currentOperator = new DataSourceScanOperator(

+				variables, dataSource);

 

-        // set empty tuple source operator

-        ILogicalOperator ets = new EmptyTupleSourceOperator();

-        currentOperator.getInputs().add(new MutableObject<ILogicalOperator>(ets));

+		// set empty tuple source operator

+		ILogicalOperator ets = new EmptyTupleSourceOperator();

+		currentOperator.getInputs().add(

+				new MutableObject<ILogicalOperator>(ets));

 

-        // setup data source

-        dataSourceMap.put(partDesc, dataSource);

-        t.rewriteOperatorOutputSchema(variables, operator);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		// setup data source

+		dataSourceMap.put(partDesc, dataSource);

+		t.rewriteOperatorOutputSchema(variables, operator);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) {

+	@Override

+	public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) {

 

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

-            return null;

+		if (hiveOperator.getChildOperators() != null

+				&& hiveOperator.getChildOperators().size() > 0)

+			return null;

 

-        Schema currentSchema = t.generateInputSchema(hiveOperator.getParentOperators().get(0));

+		Schema currentSchema = t.generateInputSchema(hiveOperator

+				.getParentOperators().get(0));

 

-        IDataSink sink = new HiveDataSink(hiveOperator, currentSchema.getSchema());

-        List<Mutable<ILogicalExpression> > exprList = new ArrayList<Mutable<ILogicalExpression>>();

-        for (String column : currentSchema.getNames()) {

-            exprList.add(new MutableObject<ILogicalExpression> (new VariableReferenceExpression(t.getVariable(column))));

-        }

+		IDataSink sink = new HiveDataSink(hiveOperator,

+				currentSchema.getSchema());

+		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();

+		for (String column : currentSchema.getNames()) {

+			exprList.add(new MutableObject<ILogicalExpression>(

+					new VariableReferenceExpression(t.getVariable(column))));

+		}

 

-        ILogicalOperator currentOperator = new WriteOperator(exprList, sink);

-        if (AlgebricksParentOperator != null) {

-            currentOperator.getInputs().add(AlgebricksParentOperator);

-        }

+		ILogicalOperator currentOperator = new WriteOperator(exprList, sink);

+		if (AlgebricksParentOperator != null) {

+			currentOperator.getInputs().add(AlgebricksParentOperator);

+		}

 

-        IMetadataProvider<PartitionDesc, Object> metaData = new HiveMetaDataProvider<PartitionDesc, Object>(

-                hiveOperator, currentSchema, dataSourceMap);

-        t.setMetadataProvider(metaData);

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		IMetadataProvider<PartitionDesc, Object> metaData = new HiveMetaDataProvider<PartitionDesc, Object>(

+				hiveOperator, currentSchema, dataSourceMap);

+		t.setMetadataProvider(metaData);

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
index 96b9463..f4e74f6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/UnionVisitor.java
@@ -18,45 +18,47 @@
 

 public class UnionVisitor extends DefaultVisitor {

 

-    List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();

+	List<Mutable<ILogicalOperator>> parents = new ArrayList<Mutable<ILogicalOperator>>();

 

-    @Override

-    public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

+	@Override

+	public Mutable<ILogicalOperator> visit(UnionOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

 

-        parents.add(AlgebricksParentOperator);

-        if (operator.getParentOperators().size() > parents.size()) {

-            return null;

-        }

+		parents.add(AlgebricksParentOperator);

+		if (operator.getParentOperators().size() > parents.size()) {

+			return null;

+		}

 

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

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

+		List<LogicalVariable> leftVars = new ArrayList<LogicalVariable>();

+		List<LogicalVariable> rightVars = new ArrayList<LogicalVariable>();

 

-        VariableUtilities.getUsedVariables(parents.get(0).getValue(), leftVars);

-        VariableUtilities.getUsedVariables(parents.get(1).getValue(), rightVars);

+		VariableUtilities.getUsedVariables(parents.get(0).getValue(), leftVars);

+		VariableUtilities

+				.getUsedVariables(parents.get(1).getValue(), rightVars);

 

-        List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> triples = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();

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

+		List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> triples = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();

+		List<LogicalVariable> unionVars = new ArrayList<LogicalVariable>();

 

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

-            LogicalVariable unionVar = t.getVariable(

-                    leftVars.get(i).getId() + "union" + AlgebricksParentOperator.hashCode(),

-                    TypeInfoFactory.unknownTypeInfo);

-            unionVars.add(unionVar);

-            Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(

-                    leftVars.get(i), rightVars.get(i), unionVar);

-            t.replaceVariable(leftVars.get(i), unionVar);

-            t.replaceVariable(rightVars.get(i), unionVar);

-            triples.add(triple);

-        }

-        ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator(

-                triples);

-        for (Mutable<ILogicalOperator> parent : parents)

-            currentOperator.getInputs().add(parent);

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

+			LogicalVariable unionVar = t.getVariable(leftVars.get(i).getId()

+					+ "union" + AlgebricksParentOperator.hashCode(),

+					TypeInfoFactory.unknownTypeInfo);

+			unionVars.add(unionVar);

+			Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(

+					leftVars.get(i), rightVars.get(i), unionVar);

+			t.replaceVariable(leftVars.get(i), unionVar);

+			t.replaceVariable(rightVars.get(i), unionVar);

+			triples.add(triple);

+		}

+		ILogicalOperator currentOperator = new edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator(

+				triples);

+		for (Mutable<ILogicalOperator> parent : parents)

+			currentOperator.getInputs().add(parent);

 

-        t.rewriteOperatorOutputSchema(unionVars, operator);

-        parents.clear();

-        return new MutableObject<ILogicalOperator>(currentOperator);

-    }

+		t.rewriteOperatorOutputSchema(unionVars, operator);

+		parents.clear();

+		return new MutableObject<ILogicalOperator>(currentOperator);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java
index 972deed..20013e3 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/DefaultVisitor.java
@@ -31,115 +31,136 @@
  */

 public class DefaultVisitor implements Visitor {

 

-    @Override

-    public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(LateralViewForwardOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(

+			LateralViewForwardOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(LateralViewJoinOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(

+			LateralViewJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(MapOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(MapOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(UDTFOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(UDTFOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperator,

-            Translator t) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public Mutable<ILogicalOperator> visit(UnionOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperator, Translator t)

+			throws AlgebricksException {

+		return null;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
index da87f53..9165386 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Translator.java
@@ -17,153 +17,158 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;

 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;

 

+@SuppressWarnings("rawtypes")

 public interface Translator {

 

-    /**

-     * generate input schema

-     * 

-     * @param operator

-     * @return

-     */

-    public Schema generateInputSchema(Operator operator);

+	/**

+	 * generate input schema

+	 * 

+	 * @param operator

+	 * @return

+	 */

+	public Schema generateInputSchema(Operator operator);

 

-    /**

-     * rewrite the names of output columns for feture expression evaluators to

-     * use

-     * 

-     * @param operator

-     */

-    public void rewriteOperatorOutputSchema(List<LogicalVariable> vars, Operator operator);

+	/**

+	 * rewrite the names of output columns for feture expression evaluators to

+	 * use

+	 * 

+	 * @param operator

+	 */

+	public void rewriteOperatorOutputSchema(List<LogicalVariable> vars,

+			Operator operator);

 

-    /**

-     * rewrite the names of output columns for feture expression evaluators to

-     * use

-     * 

-     * @param operator

-     */

-    public void rewriteOperatorOutputSchema(Operator operator);

+	/**

+	 * rewrite the names of output columns for feture expression evaluators to

+	 * use

+	 * 

+	 * @param operator

+	 */

+	public void rewriteOperatorOutputSchema(Operator operator);

 

-    /**

-     * rewrite an expression and substitute variables

-     * 

-     * @param expr

-     *            hive expression

-     */

-    public void rewriteExpression(ExprNodeDesc expr);

+	/**

+	 * rewrite an expression and substitute variables

+	 * 

+	 * @param expr

+	 *            hive expression

+	 */

+	public void rewriteExpression(ExprNodeDesc expr);

 

-    /**

-     * rewrite an expression and substitute variables

-     * 

-     * @param expr

-     *            hive expression

-     */

-    public void rewriteExpressionPartial(ExprNodeDesc expr);

+	/**

+	 * rewrite an expression and substitute variables

+	 * 

+	 * @param expr

+	 *            hive expression

+	 */

+	public void rewriteExpressionPartial(ExprNodeDesc expr);

 

-    /**

-     * get an assign operator as a child of parent

-     * 

-     * @param parent

-     * @param cols

-     * @param variables

-     * @return

-     */

-    public ILogicalOperator getAssignOperator(Mutable<ILogicalOperator> parent, List<ExprNodeDesc> cols,

-            ArrayList<LogicalVariable> variables);

+	/**

+	 * get an assign operator as a child of parent

+	 * 

+	 * @param parent

+	 * @param cols

+	 * @param variables

+	 * @return

+	 */

+	public ILogicalOperator getAssignOperator(Mutable<ILogicalOperator> parent,

+			List<ExprNodeDesc> cols, ArrayList<LogicalVariable> variables);

 

-    /**

-     * get type for a logical variable

-     * 

-     * @param var

-     * @return type info

-     */

-    public TypeInfo getType(LogicalVariable var);

+	/**

+	 * get type for a logical variable

+	 * 

+	 * @param var

+	 * @return type info

+	 */

+	public TypeInfo getType(LogicalVariable var);

 

-    /**

-     * translate an expression from hive to Algebricks

-     * 

-     * @param desc

-     * @return

-     */

-    public Mutable<ILogicalExpression> translateScalarFucntion(ExprNodeDesc desc);

+	/**

+	 * translate an expression from hive to Algebricks

+	 * 

+	 * @param desc

+	 * @return

+	 */

+	public Mutable<ILogicalExpression> translateScalarFucntion(ExprNodeDesc desc);

 

-    /**

-     * translate an aggregation from hive to Algebricks

-     * 

-     * @param aggregateDesc

-     * @return

-     */

-    public Mutable<ILogicalExpression> translateAggregation(AggregationDesc aggregateDesc);

+	/**

+	 * translate an aggregation from hive to Algebricks

+	 * 

+	 * @param aggregateDesc

+	 * @return

+	 */

+	public Mutable<ILogicalExpression> translateAggregation(

+			AggregationDesc aggregateDesc);

 

-    /**

-     * translate unnesting (UDTF) function expression

-     * 

-     * @param aggregator

-     * @return

-     */

-    public Mutable<ILogicalExpression> translateUnnestFunction(UDTFDesc udtfDesc, Mutable<ILogicalExpression> argument);

+	/**

+	 * translate unnesting (UDTF) function expression

+	 * 

+	 * @param aggregator

+	 * @return

+	 */

+	public Mutable<ILogicalExpression> translateUnnestFunction(

+			UDTFDesc udtfDesc, Mutable<ILogicalExpression> argument);

 

-    /**

-     * get variable from a schema

-     * 

-     * @param schema

-     * @return

-     */

-    public List<LogicalVariable> getVariablesFromSchema(Schema schema);

+	/**

+	 * get variable from a schema

+	 * 

+	 * @param schema

+	 * @return

+	 */

+	public List<LogicalVariable> getVariablesFromSchema(Schema schema);

 

-    /**

-     * get variable from name

-     * 

-     * @param name

-     * @return

-     */

-    public LogicalVariable getVariable(String name);

+	/**

+	 * get variable from name

+	 * 

+	 * @param name

+	 * @return

+	 */

+	public LogicalVariable getVariable(String name);

 

-    /**

-     * get variable from field name

-     * 

-     * @param name

-     * @return

-     */

-    public LogicalVariable getVariableFromFieldName(String name);

+	/**

+	 * get variable from field name

+	 * 

+	 * @param name

+	 * @return

+	 */

+	public LogicalVariable getVariableFromFieldName(String name);

 

-    /**

-     * get variable from name, type

-     * 

-     * @param fieldName

-     * @param type

-     * @return

-     */

-    public LogicalVariable getVariable(String fieldName, TypeInfo type);

+	/**

+	 * get variable from name, type

+	 * 

+	 * @param fieldName

+	 * @param type

+	 * @return

+	 */

+	public LogicalVariable getVariable(String fieldName, TypeInfo type);

 

-    /**

-     * get new variable from name, type

-     * 

-     * @param fieldName

-     * @param type

-     * @return

-     */

-    public LogicalVariable getNewVariable(String fieldName, TypeInfo type);

+	/**

+	 * get new variable from name, type

+	 * 

+	 * @param fieldName

+	 * @param type

+	 * @return

+	 */

+	public LogicalVariable getNewVariable(String fieldName, TypeInfo type);

 

-    /**

-     * set the metadata provider

-     * 

-     * @param metadata

-     */

-    public void setMetadataProvider(IMetadataProvider<PartitionDesc, Object> metadata);

+	/**

+	 * set the metadata provider

+	 * 

+	 * @param metadata

+	 */

+	public void setMetadataProvider(

+			IMetadataProvider<PartitionDesc, Object> metadata);

 

-    /**

-     * get the metadata provider

-     * 

-     * @param metadata

-     */

-    public IMetadataProvider<PartitionDesc, Object> getMetadataProvider();

+	/**

+	 * get the metadata provider

+	 * 

+	 * @param metadata

+	 */

+	public IMetadataProvider<PartitionDesc, Object> getMetadataProvider();

 

-    /**

-     * replace the variable

-     * 

-     * @param oldVar

-     * @param newVar

-     */

-    public void replaceVariable(LogicalVariable oldVar, LogicalVariable newVar);

+	/**

+	 * replace the variable

+	 * 

+	 * @param oldVar

+	 * @param newVar

+	 */

+	public void replaceVariable(LogicalVariable oldVar, LogicalVariable newVar);

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java
index 495f3ee..745f93e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/logical/plan/visitor/base/Visitor.java
@@ -26,60 +26,81 @@
 

 public interface Visitor {

 

-    public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(CollectOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(JoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(ExtractOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(MapJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(SMBMapJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(FilterOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(ForwardOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(GroupByOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(LateralViewForwardOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(

+			LateralViewForwardOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(LateralViewJoinOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(

+			LateralViewJoinOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(LimitOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(MapOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(MapOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(ScriptOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(SelectOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(TableScanOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(FileSinkOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator,

-            Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(ReduceSinkOperator hiveOperator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(UDTFOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(UDTFOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 

-    public Mutable<ILogicalOperator> visit(UnionOperator operator, Mutable<ILogicalOperator> AlgebricksParentOperatorRef,

-            Translator t) throws AlgebricksException;

+	public Mutable<ILogicalOperator> visit(UnionOperator operator,

+			Mutable<ILogicalOperator> AlgebricksParentOperatorRef, Translator t)

+			throws AlgebricksException;

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
index 7e4e271..4ebea0a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
@@ -37,77 +37,78 @@
 

 public final class HiveRuleCollections {

 

-    public final static LinkedList<IAlgebraicRewriteRule> NORMALIZATION = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        NORMALIZATION.add(new EliminateSubplanRule());

-        NORMALIZATION.add(new IntroduceAggregateCombinerRule());

-        NORMALIZATION.add(new BreakSelectIntoConjunctsRule());

-        NORMALIZATION.add(new IntroduceAggregateCombinerRule());

-        NORMALIZATION.add(new PushSelectIntoJoinRule());

-        NORMALIZATION.add(new ExtractGbyExpressionsRule());

-        NORMALIZATION.add(new RemoveRedundantSelectRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> NORMALIZATION = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		NORMALIZATION.add(new EliminateSubplanRule());

+		NORMALIZATION.add(new IntroduceAggregateCombinerRule());

+		NORMALIZATION.add(new BreakSelectIntoConjunctsRule());

+		NORMALIZATION.add(new IntroduceAggregateCombinerRule());

+		NORMALIZATION.add(new PushSelectIntoJoinRule());

+		NORMALIZATION.add(new ExtractGbyExpressionsRule());

+		NORMALIZATION.add(new RemoveRedundantSelectRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> COND_PUSHDOWN_AND_JOIN_INFERENCE = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new PushSelectDownRule());

-        COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new InlineVariablesRule());

-        COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new FactorRedundantGroupAndDecorVarsRule());

-        COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new EliminateSubplanRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> COND_PUSHDOWN_AND_JOIN_INFERENCE = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new PushSelectDownRule());

+		COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new InlineVariablesRule());

+		COND_PUSHDOWN_AND_JOIN_INFERENCE

+				.add(new FactorRedundantGroupAndDecorVarsRule());

+		COND_PUSHDOWN_AND_JOIN_INFERENCE.add(new EliminateSubplanRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> LOAD_FIELDS = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        // should LoadRecordFieldsRule be applied in only one pass over the

-        // plan?

-        LOAD_FIELDS.add(new InlineVariablesRule());

-        // LOAD_FIELDS.add(new RemoveUnusedAssignAndAggregateRule());

-        LOAD_FIELDS.add(new ComplexJoinInferenceRule());

-        LOAD_FIELDS.add(new InferTypesRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> LOAD_FIELDS = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		// should LoadRecordFieldsRule be applied in only one pass over the

+		// plan?

+		LOAD_FIELDS.add(new InlineVariablesRule());

+		// LOAD_FIELDS.add(new RemoveUnusedAssignAndAggregateRule());

+		LOAD_FIELDS.add(new ComplexJoinInferenceRule());

+		LOAD_FIELDS.add(new InferTypesRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> OP_PUSHDOWN = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        OP_PUSHDOWN.add(new PushProjectDownRule());

-        OP_PUSHDOWN.add(new PushSelectDownRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> OP_PUSHDOWN = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		OP_PUSHDOWN.add(new PushProjectDownRule());

+		OP_PUSHDOWN.add(new PushSelectDownRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> DATA_EXCHANGE = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        DATA_EXCHANGE.add(new SetExecutionModeRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> DATA_EXCHANGE = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		DATA_EXCHANGE.add(new SetExecutionModeRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> CONSOLIDATION = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        CONSOLIDATION.add(new RemoveRedundantProjectionRule());

-        CONSOLIDATION.add(new ConsolidateSelectsRule());

-        CONSOLIDATION.add(new IntroduceEarlyProjectRule());

-        CONSOLIDATION.add(new ConsolidateAssignsRule());

-        CONSOLIDATION.add(new IntroduceGroupByCombinerRule());

-        CONSOLIDATION.add(new RemoveUnusedAssignAndAggregateRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> CONSOLIDATION = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		CONSOLIDATION.add(new RemoveRedundantProjectionRule());

+		CONSOLIDATION.add(new ConsolidateSelectsRule());

+		CONSOLIDATION.add(new IntroduceEarlyProjectRule());

+		CONSOLIDATION.add(new ConsolidateAssignsRule());

+		CONSOLIDATION.add(new IntroduceGroupByCombinerRule());

+		CONSOLIDATION.add(new RemoveUnusedAssignAndAggregateRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> PHYSICAL_PLAN_REWRITES = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        PHYSICAL_PLAN_REWRITES.add(new PullSelectOutOfEqJoin());

-        PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());

-        PHYSICAL_PLAN_REWRITES.add(new EnforceStructuralPropertiesRule());

-        PHYSICAL_PLAN_REWRITES.add(new PushProjectDownRule());

-        PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());

-        PHYSICAL_PLAN_REWRITES.add(new PushLimitDownRule());

-        PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeWriteRule());

-        PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeUnionRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> PHYSICAL_PLAN_REWRITES = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		PHYSICAL_PLAN_REWRITES.add(new PullSelectOutOfEqJoin());

+		PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());

+		PHYSICAL_PLAN_REWRITES.add(new EnforceStructuralPropertiesRule());

+		PHYSICAL_PLAN_REWRITES.add(new PushProjectDownRule());

+		PHYSICAL_PLAN_REWRITES.add(new SetAlgebricksPhysicalOperatorsRule());

+		PHYSICAL_PLAN_REWRITES.add(new PushLimitDownRule());

+		PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeWriteRule());

+		PHYSICAL_PLAN_REWRITES.add(new InsertProjectBeforeUnionRule());

+	}

 

-    public final static LinkedList<IAlgebraicRewriteRule> prepareJobGenRules = new LinkedList<IAlgebraicRewriteRule>();

-    static {

-        prepareJobGenRules.add(new ReinferAllTypesRule());

-        prepareJobGenRules.add(new IsolateHyracksOperatorsRule(

-                HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));

-        prepareJobGenRules.add(new ExtractCommonOperatorsRule());

-        prepareJobGenRules.add(new LocalGroupByRule());

-        prepareJobGenRules.add(new PushProjectIntoDataSourceScanRule());

-        prepareJobGenRules.add(new ReinferAllTypesRule());

-    }

+	public final static LinkedList<IAlgebraicRewriteRule> prepareJobGenRules = new LinkedList<IAlgebraicRewriteRule>();

+	static {

+		prepareJobGenRules.add(new ReinferAllTypesRule());

+		prepareJobGenRules.add(new IsolateHyracksOperatorsRule(

+				HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));

+		prepareJobGenRules.add(new ExtractCommonOperatorsRule());

+		prepareJobGenRules.add(new LocalGroupByRule());

+		prepareJobGenRules.add(new PushProjectIntoDataSourceScanRule());

+		prepareJobGenRules.add(new ReinferAllTypesRule());

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java
index 90777ee..c58982e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/InsertProjectBeforeWriteRule.java
@@ -21,59 +21,65 @@
 
 public class InsertProjectBeforeWriteRule implements IAlgebraicRewriteRule {
 
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
+	@Override
+	public boolean rewritePost(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) {
+		return false;
+	}
 
-    /**
-     * When the input schema to WriteOperator is different from the output
-     * schema in terms of variable order, add a project operator to get the
-     * write order
-     */
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.WRITE) {
-            return false;
-        }
-        WriteOperator opWrite = (WriteOperator) op;
-        ArrayList<LogicalVariable> finalSchema = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(opWrite, finalSchema);
-        ArrayList<LogicalVariable> inputSchema = new ArrayList<LogicalVariable>();
-        VariableUtilities.getLiveVariables(opWrite, inputSchema);
-        if (!isIdentical(finalSchema, inputSchema)) {
-            ProjectOperator projectOp = new ProjectOperator(finalSchema);
-            Mutable<ILogicalOperator> parentOpRef = opWrite.getInputs().get(0);
-            projectOp.getInputs().add(parentOpRef);
-            opWrite.getInputs().clear();
-            opWrite.getInputs().add(new MutableObject<ILogicalOperator>(projectOp));
-            projectOp.setPhysicalOperator(new StreamProjectPOperator());
-            projectOp.setExecutionMode(ExecutionMode.PARTITIONED);
+	/**
+	 * When the input schema to WriteOperator is different from the output
+	 * schema in terms of variable order, add a project operator to get the
+	 * write order
+	 */
+	@Override
+	public boolean rewritePre(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+		if (op.getOperatorTag() != LogicalOperatorTag.WRITE) {
+			return false;
+		}
+		WriteOperator opWrite = (WriteOperator) op;
+		ArrayList<LogicalVariable> finalSchema = new ArrayList<LogicalVariable>();
+		VariableUtilities.getUsedVariables(opWrite, finalSchema);
+		ArrayList<LogicalVariable> inputSchema = new ArrayList<LogicalVariable>();
+		VariableUtilities.getLiveVariables(opWrite, inputSchema);
+		if (!isIdentical(finalSchema, inputSchema)) {
+			ProjectOperator projectOp = new ProjectOperator(finalSchema);
+			Mutable<ILogicalOperator> parentOpRef = opWrite.getInputs().get(0);
+			projectOp.getInputs().add(parentOpRef);
+			opWrite.getInputs().clear();
+			opWrite.getInputs().add(
+					new MutableObject<ILogicalOperator>(projectOp));
+			projectOp.setPhysicalOperator(new StreamProjectPOperator());
+			projectOp.setExecutionMode(ExecutionMode.PARTITIONED);
 
-            AbstractLogicalOperator op2 = (AbstractLogicalOperator) parentOpRef.getValue();
-            if (op2.getOperatorTag() == LogicalOperatorTag.PROJECT) {
-                ProjectOperator pi2 = (ProjectOperator) op2;
-                parentOpRef.setValue(pi2.getInputs().get(0).getValue());
-            }
-            context.computeAndSetTypeEnvironmentForOperator(projectOp);
-            return true;
-        } else
-            return false;
+			AbstractLogicalOperator op2 = (AbstractLogicalOperator) parentOpRef
+					.getValue();
+			if (op2.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+				ProjectOperator pi2 = (ProjectOperator) op2;
+				parentOpRef.setValue(pi2.getInputs().get(0).getValue());
+			}
+			context.computeAndSetTypeEnvironmentForOperator(projectOp);
+			return true;
+		} else
+			return false;
 
-    }
+	}
 
-    private boolean isIdentical(List<LogicalVariable> finalSchema, List<LogicalVariable> inputSchema) {
-        int finalSchemaSize = finalSchema.size();
-        int inputSchemaSize = inputSchema.size();
-        if (finalSchemaSize != inputSchemaSize)
-            throw new IllegalStateException("final output schema variables missing!");
-        for (int i = 0; i < finalSchemaSize; i++) {
-            LogicalVariable var1 = finalSchema.get(i);
-            LogicalVariable var2 = inputSchema.get(i);
-            if (!var1.equals(var2))
-                return false;
-        }
-        return true;
-    }
+	private boolean isIdentical(List<LogicalVariable> finalSchema,
+			List<LogicalVariable> inputSchema) {
+		int finalSchemaSize = finalSchema.size();
+		int inputSchemaSize = inputSchema.size();
+		if (finalSchemaSize != inputSchemaSize)
+			throw new IllegalStateException(
+					"final output schema variables missing!");
+		for (int i = 0; i < finalSchemaSize; i++) {
+			LogicalVariable var1 = finalSchema.get(i);
+			LogicalVariable var2 = inputSchema.get(i);
+			if (!var1.equals(var2))
+				return false;
+		}
+		return true;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java
index 96815ff..2bebe81 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/IntroduceEarlyProjectRule.java
@@ -20,53 +20,58 @@
 
 public class IntroduceEarlyProjectRule implements IAlgebraicRewriteRule {
 
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
-    }
+	@Override
+	public boolean rewritePre(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		return false;
+	}
 
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
-            return false;
-        }
-        AbstractLogicalOperator middleOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        List<LogicalVariable> deliveredVars = new ArrayList<LogicalVariable>();
-        List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
-        List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+	@Override
+	public boolean rewritePost(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+		if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+			return false;
+		}
+		AbstractLogicalOperator middleOp = (AbstractLogicalOperator) op
+				.getInputs().get(0).getValue();
+		List<LogicalVariable> deliveredVars = new ArrayList<LogicalVariable>();
+		List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+		List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
 
-        VariableUtilities.getUsedVariables(op, deliveredVars);
-        VariableUtilities.getUsedVariables(middleOp, usedVars);
-        VariableUtilities.getProducedVariables(middleOp, producedVars);
+		VariableUtilities.getUsedVariables(op, deliveredVars);
+		VariableUtilities.getUsedVariables(middleOp, usedVars);
+		VariableUtilities.getProducedVariables(middleOp, producedVars);
 
-        Set<LogicalVariable> requiredVariables = new HashSet<LogicalVariable>();
-        requiredVariables.addAll(deliveredVars);
-        requiredVariables.addAll(usedVars);
-        requiredVariables.removeAll(producedVars);
+		Set<LogicalVariable> requiredVariables = new HashSet<LogicalVariable>();
+		requiredVariables.addAll(deliveredVars);
+		requiredVariables.addAll(usedVars);
+		requiredVariables.removeAll(producedVars);
 
-        if (middleOp.getInputs().size() <= 0 || middleOp.getInputs().size() > 1)
-            return false;
+		if (middleOp.getInputs().size() <= 0 || middleOp.getInputs().size() > 1)
+			return false;
 
-        AbstractLogicalOperator targetOp = (AbstractLogicalOperator) middleOp.getInputs().get(0).getValue();
-        if (targetOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
-            return false;
+		AbstractLogicalOperator targetOp = (AbstractLogicalOperator) middleOp
+				.getInputs().get(0).getValue();
+		if (targetOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
+			return false;
 
-        Set<LogicalVariable> deliveredEarlyVars = new HashSet<LogicalVariable>();
-        VariableUtilities.getLiveVariables(targetOp, deliveredEarlyVars);
+		Set<LogicalVariable> deliveredEarlyVars = new HashSet<LogicalVariable>();
+		VariableUtilities.getLiveVariables(targetOp, deliveredEarlyVars);
 
-        deliveredEarlyVars.removeAll(requiredVariables);
-        if (deliveredEarlyVars.size() > 0) {
-            ArrayList<LogicalVariable> requiredVars = new ArrayList<LogicalVariable>();
-            requiredVars.addAll(requiredVariables);
-            ILogicalOperator earlyProjectOp = new ProjectOperator(requiredVars);
-            Mutable<ILogicalOperator> earlyProjectOpRef = new MutableObject<ILogicalOperator>(earlyProjectOp);
-            Mutable<ILogicalOperator> targetRef = middleOp.getInputs().get(0);
-            middleOp.getInputs().set(0, earlyProjectOpRef);
-            earlyProjectOp.getInputs().add(targetRef);
-            context.computeAndSetTypeEnvironmentForOperator(earlyProjectOp);
-            return true;
-        }
-        return false;
-    }
+		deliveredEarlyVars.removeAll(requiredVariables);
+		if (deliveredEarlyVars.size() > 0) {
+			ArrayList<LogicalVariable> requiredVars = new ArrayList<LogicalVariable>();
+			requiredVars.addAll(requiredVariables);
+			ILogicalOperator earlyProjectOp = new ProjectOperator(requiredVars);
+			Mutable<ILogicalOperator> earlyProjectOpRef = new MutableObject<ILogicalOperator>(
+					earlyProjectOp);
+			Mutable<ILogicalOperator> targetRef = middleOp.getInputs().get(0);
+			middleOp.getInputs().set(0, earlyProjectOpRef);
+			earlyProjectOp.getInputs().add(targetRef);
+			context.computeAndSetTypeEnvironmentForOperator(earlyProjectOp);
+			return true;
+		}
+		return false;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java
index 90ca008..72cbe21 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/LocalGroupByRule.java
@@ -17,50 +17,55 @@
 
 public class LocalGroupByRule implements IAlgebraicRewriteRule {
 
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
-    }
+	@Override
+	public boolean rewritePre(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		return false;
+	}
 
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
-            return false;
-        }
-        Boolean localGby = (Boolean) op.getAnnotations().get(HiveOperatorAnnotations.LOCAL_GROUP_BY);
-        if (localGby != null && localGby.equals(Boolean.TRUE)) {
-            Boolean hashGby = (Boolean) op.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY);
-            Boolean externalGby = (Boolean) op.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY);
-            if ((hashGby != null && (hashGby.equals(Boolean.TRUE)) || (externalGby != null && externalGby
-                    .equals(Boolean.TRUE)))) {
-                reviseExchange(op);
-            } else {
-                ILogicalOperator child = op.getInputs().get(0).getValue();
-                AbstractLogicalOperator childOp = (AbstractLogicalOperator) child;
-                while (child.getInputs().size() > 0) {
-                    if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
-                        break;
-                    else {
-                        child = child.getInputs().get(0).getValue();
-                        childOp = (AbstractLogicalOperator) child;
-                    }
-                }
-                if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
-                    reviseExchange(childOp);
-            }
-            return true;
-        }
-        return false;
-    }
+	@Override
+	public boolean rewritePost(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+		if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+			return false;
+		}
+		Boolean localGby = (Boolean) op.getAnnotations().get(
+				HiveOperatorAnnotations.LOCAL_GROUP_BY);
+		if (localGby != null && localGby.equals(Boolean.TRUE)) {
+			Boolean hashGby = (Boolean) op.getAnnotations().get(
+					OperatorAnnotations.USE_HASH_GROUP_BY);
+			Boolean externalGby = (Boolean) op.getAnnotations().get(
+					OperatorAnnotations.USE_EXTERNAL_GROUP_BY);
+			if ((hashGby != null && (hashGby.equals(Boolean.TRUE)) || (externalGby != null && externalGby
+					.equals(Boolean.TRUE)))) {
+				reviseExchange(op);
+			} else {
+				ILogicalOperator child = op.getInputs().get(0).getValue();
+				AbstractLogicalOperator childOp = (AbstractLogicalOperator) child;
+				while (child.getInputs().size() > 0) {
+					if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
+						break;
+					else {
+						child = child.getInputs().get(0).getValue();
+						childOp = (AbstractLogicalOperator) child;
+					}
+				}
+				if (childOp.getOperatorTag() == LogicalOperatorTag.ORDER)
+					reviseExchange(childOp);
+			}
+			return true;
+		}
+		return false;
+	}
 
-    private void reviseExchange(AbstractLogicalOperator op) {
-        ExchangeOperator exchange = (ExchangeOperator) op.getInputs().get(0).getValue();
-        IPhysicalOperator physicalOp = exchange.getPhysicalOperator();
-        if (physicalOp.getOperatorTag() == PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
-            exchange.setPhysicalOperator(new OneToOneExchangePOperator());
-        }
-    }
+	private void reviseExchange(AbstractLogicalOperator op) {
+		ExchangeOperator exchange = (ExchangeOperator) op.getInputs().get(0)
+				.getValue();
+		IPhysicalOperator physicalOp = exchange.getPhysicalOperator();
+		if (physicalOp.getOperatorTag() == PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
+			exchange.setPhysicalOperator(new OneToOneExchangePOperator());
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java
index 44ff12d..9958ba8 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/optimizer/rules/RemoveRedundantSelectRule.java
@@ -13,32 +13,34 @@
 
 public class RemoveRedundantSelectRule implements IAlgebraicRewriteRule {
 
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
-    }
+	@Override
+	public boolean rewritePre(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		return false;
+	}
 
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
-        }
-        AbstractLogicalOperator inputOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        if (inputOp.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
-        }
-        SelectOperator selectOp = (SelectOperator) op;
-        SelectOperator inputSelectOp = (SelectOperator) inputOp;
-        ILogicalExpression expr1 = selectOp.getCondition().getValue();
-        ILogicalExpression expr2 = inputSelectOp.getCondition().getValue();
+	@Override
+	public boolean rewritePost(Mutable<ILogicalOperator> opRef,
+			IOptimizationContext context) throws AlgebricksException {
+		AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+		if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+			return false;
+		}
+		AbstractLogicalOperator inputOp = (AbstractLogicalOperator) op
+				.getInputs().get(0).getValue();
+		if (inputOp.getOperatorTag() != LogicalOperatorTag.SELECT) {
+			return false;
+		}
+		SelectOperator selectOp = (SelectOperator) op;
+		SelectOperator inputSelectOp = (SelectOperator) inputOp;
+		ILogicalExpression expr1 = selectOp.getCondition().getValue();
+		ILogicalExpression expr2 = inputSelectOp.getCondition().getValue();
 
-        if (expr1.equals(expr2)) {
-            selectOp.getInputs().set(0, inputSelectOp.getInputs().get(0));
-            return true;
-        }
-        return false;
-    }
+		if (expr1.equals(expr2)) {
+			selectOp.getInputs().set(0, inputSelectOp.getInputs().get(0));
+			return true;
+		}
+		return false;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java
index 10596b6..6b4d697 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/config/ConfUtil.java
@@ -19,120 +19,126 @@
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;

 import edu.uci.ics.hyracks.api.topology.ClusterTopology;

 

+@SuppressWarnings({ "rawtypes", "deprecation" })

 public class ConfUtil {

 

-    private static JobConf job;

-    private static HiveConf hconf;

-    private static String[] NCs;

-    private static Map<String, List<String>> ncMapping;

-    private static IHyracksClientConnection hcc = null;

-    private static ClusterTopology topology = null;

+	private static JobConf job;

+	private static HiveConf hconf;

+	private static String[] NCs;

+	private static Map<String, List<String>> ncMapping;

+	private static IHyracksClientConnection hcc = null;

+	private static ClusterTopology topology = null;

 

-    public static JobConf getJobConf(Class<? extends InputFormat> format, Path path) {

-        JobConf conf = new JobConf();

-        if (job != null)

-            conf = job;

+	public static JobConf getJobConf(Class<? extends InputFormat> format,

+			Path path) {

+		JobConf conf = new JobConf();

+		if (job != null)

+			conf = job;

 

-        String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");

-        Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");

-        conf.addResource(pathCore);

-        Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");

-        conf.addResource(pathMapRed);

-        Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");

-        conf.addResource(pathHDFS);

+		String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");

+		Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");

+		conf.addResource(pathCore);

+		Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");

+		conf.addResource(pathMapRed);

+		Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");

+		conf.addResource(pathHDFS);

 

-        conf.setInputFormat(format);

-        FileInputFormat.setInputPaths(conf, path);

-        return conf;

-    }

+		conf.setInputFormat(format);

+		FileInputFormat.setInputPaths(conf, path);

+		return conf;

+	}

 

-    public static JobConf getJobConf() {

-        JobConf conf = new JobConf();

-        if (job != null)

-            conf = job;

+	public static JobConf getJobConf() {

+		JobConf conf = new JobConf();

+		if (job != null)

+			conf = job;

 

-        String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");

-        Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");

-        conf.addResource(pathCore);

-        Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");

-        conf.addResource(pathMapRed);

-        Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");

-        conf.addResource(pathHDFS);

+		String hadoopPath = System.getProperty("HADOOP_HOME", "/hadoop");

+		Path pathCore = new Path(hadoopPath + "/conf/core-site.xml");

+		conf.addResource(pathCore);

+		Path pathMapRed = new Path(hadoopPath + "/conf/mapred-site.xml");

+		conf.addResource(pathMapRed);

+		Path pathHDFS = new Path(hadoopPath + "/conf/hdfs-site.xml");

+		conf.addResource(pathHDFS);

 

-        return conf;

-    }

+		return conf;

+	}

 

-    public static void setJobConf(JobConf conf) {

-        job = conf;

-    }

+	public static void setJobConf(JobConf conf) {

+		job = conf;

+	}

 

-    public static void setHiveConf(HiveConf hiveConf) {

-        hconf = hiveConf;

-    }

+	public static void setHiveConf(HiveConf hiveConf) {

+		hconf = hiveConf;

+	}

 

-    public static HiveConf getHiveConf() {

-        if (hconf == null) {

-            hconf = new HiveConf(SessionState.class);

-            hconf.addResource(new Path("conf/hive-default.xml"));

-        }

-        return hconf;

-    }

+	public static HiveConf getHiveConf() {

+		if (hconf == null) {

+			hconf = new HiveConf(SessionState.class);

+			hconf.addResource(new Path("conf/hive-default.xml"));

+		}

+		return hconf;

+	}

 

-    public static String[] getNCs() throws AlgebricksException {

-        if (NCs == null) {

-            try {

-                loadClusterConfig();

-            } catch (Exception e) {

-                throw new AlgebricksException(e);

-            }

-        }

-        return NCs;

-    }

+	public static String[] getNCs() throws AlgebricksException {

+		if (NCs == null) {

+			try {

+				loadClusterConfig();

+			} catch (Exception e) {

+				throw new AlgebricksException(e);

+			}

+		}

+		return NCs;

+	}

 

-    public static Map<String, List<String>> getNCMapping() throws AlgebricksException {

-        if (ncMapping == null) {

-            try {

-                loadClusterConfig();

-            } catch (Exception e) {

-                throw new AlgebricksException(e);

-            }

-        }

-        return ncMapping;

-    }

+	public static Map<String, List<String>> getNCMapping()

+			throws AlgebricksException {

+		if (ncMapping == null) {

+			try {

+				loadClusterConfig();

+			} catch (Exception e) {

+				throw new AlgebricksException(e);

+			}

+		}

+		return ncMapping;

+	}

 

-    private static void loadClusterConfig() {

-        try {

-            getHiveConf();

-            String ipAddress = hconf.get("hive.hyracks.host");

-            int port = Integer.parseInt(hconf.get("hive.hyracks.port"));

-            int mpl = Integer.parseInt(hconf.get("hive.hyracks.parrallelism"));

-            hcc = new HyracksConnection(ipAddress, port);

-            topology = hcc.getClusterTopology();

-            Map<String, NodeControllerInfo> ncNameToNcInfos = hcc.getNodeControllerInfos();

-            NCs = new String[ncNameToNcInfos.size() * mpl];

-            ncMapping = new HashMap<String, List<String>>();

-            int i = 0;

-            for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {

-                String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())

-                        .getHostAddress();

-                List<String> matchedNCs = ncMapping.get(ipAddr);

-                if (matchedNCs == null) {

-                    matchedNCs = new ArrayList<String>();

-                    ncMapping.put(ipAddr, matchedNCs);

-                }

-                matchedNCs.add(entry.getKey());

-                for (int j = i * mpl; j < i * mpl + mpl; j++)

-                    NCs[j] = entry.getKey();

-                i++;

-            }

-        } catch (Exception e) {

-            throw new IllegalStateException(e);

-        }

-    }

+	private static void loadClusterConfig() {

+		try {

+			getHiveConf();

+			String ipAddress = hconf.get("hive.hyracks.host");

+			int port = Integer.parseInt(hconf.get("hive.hyracks.port"));

+			int mpl = Integer.parseInt(hconf.get("hive.hyracks.parrallelism"));

+			hcc = new HyracksConnection(ipAddress, port);

+			topology = hcc.getClusterTopology();

+			Map<String, NodeControllerInfo> ncNameToNcInfos = hcc

+					.getNodeControllerInfos();

+			NCs = new String[ncNameToNcInfos.size() * mpl];

+			ncMapping = new HashMap<String, List<String>>();

+			int i = 0;

+			for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos

+					.entrySet()) {

+				String ipAddr = InetAddress.getByAddress(

+						entry.getValue().getNetworkAddress().getIpAddress())

+						.getHostAddress();

+				List<String> matchedNCs = ncMapping.get(ipAddr);

+				if (matchedNCs == null) {

+					matchedNCs = new ArrayList<String>();

+					ncMapping.put(ipAddr, matchedNCs);

+				}

+				matchedNCs.add(entry.getKey());

+				for (int j = i * mpl; j < i * mpl + mpl; j++)

+					NCs[j] = entry.getKey();

+				i++;

+			}

+		} catch (Exception e) {

+			throw new IllegalStateException(e);

+		}

+	}

 

-    public static ClusterTopology getClusterTopology() {

-        if (topology == null)

-            loadClusterConfig();

-        return topology;

-    }

+	public static ClusterTopology getClusterTopology() {

+		if (topology == null)

+			loadClusterConfig();

+		return topology;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
index ad02239..8f6d9ca 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AbstractExpressionEvaluator.java
@@ -22,148 +22,153 @@
 

 public abstract class AbstractExpressionEvaluator implements ICopyEvaluator {

 

-    private List<ICopyEvaluator> children;

+	private List<ICopyEvaluator> children;

 

-    private ExprNodeEvaluator evaluator;

+	private ExprNodeEvaluator evaluator;

 

-    private IDataOutputProvider out;

+	private IDataOutputProvider out;

 

-    private ObjectInspector inspector;

+	private ObjectInspector inspector;

 

-    /**

-     * output object inspector

-     */

-    private ObjectInspector outputInspector;

+	/**

+	 * output object inspector

+	 */

+	private ObjectInspector outputInspector;

 

-    /**

-     * cached row object

-     */

-    private LazyObject<? extends ObjectInspector> cachedRowObject;

+	/**

+	 * cached row object

+	 */

+	private LazyObject<? extends ObjectInspector> cachedRowObject;

 

-    /**

-     * serializer/derialzer for lazy object

-     */

-    private SerDe lazySer;

+	/**

+	 * serializer/derialzer for lazy object

+	 */

+	private SerDe lazySer;

 

-    /**

-     * data output

-     */

-    DataOutput dataOutput;

+	/**

+	 * data output

+	 */

+	DataOutput dataOutput;

 

-    public AbstractExpressionEvaluator(ExprNodeEvaluator hiveEvaluator, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        evaluator = hiveEvaluator;

-        out = output;

-        inspector = oi;

-        dataOutput = out.getDataOutput();

-    }

+	public AbstractExpressionEvaluator(ExprNodeEvaluator hiveEvaluator,

+			ObjectInspector oi, IDataOutputProvider output)

+			throws AlgebricksException {

+		evaluator = hiveEvaluator;

+		out = output;

+		inspector = oi;

+		dataOutput = out.getDataOutput();

+	}

 

-    protected ObjectInspector getRowInspector() {

-        return null;

-    }

+	protected ObjectInspector getRowInspector() {

+		return null;

+	}

 

-    protected IDataOutputProvider getIDataOutputProvider() {

-        return out;

-    }

+	protected IDataOutputProvider getIDataOutputProvider() {

+		return out;

+	}

 

-    protected ExprNodeEvaluator getHiveEvaluator() {

-        return evaluator;

-    }

+	protected ExprNodeEvaluator getHiveEvaluator() {

+		return evaluator;

+	}

 

-    public ObjectInspector getObjectInspector() {

-        return inspector;

-    }

+	public ObjectInspector getObjectInspector() {

+		return inspector;

+	}

 

-    @Override

-    public void evaluate(IFrameTupleReference r) throws AlgebricksException {

-        // initialize hive evaluator

-        try {

-            if (outputInspector == null)

-                outputInspector = evaluator.initialize(inspector);

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e.getMessage());

-        }

+	@Override

+	public void evaluate(IFrameTupleReference r) throws AlgebricksException {

+		// initialize hive evaluator

+		try {

+			if (outputInspector == null)

+				outputInspector = evaluator.initialize(inspector);

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e.getMessage());

+		}

 

-        readIntoCache(r);

-        try {

-            Object result = evaluator.evaluate(cachedRowObject);

+		readIntoCache(r);

+		try {

+			Object result = evaluator.evaluate(cachedRowObject);

 

-            // if (result == null) {

-            // result = evaluator.evaluate(cachedRowObject);

-            //

-            // // check if result is null

-            //

-            // String errorMsg = "serialize null object in  \n output " +

-            // outputInspector.toString() + " \n input "

-            // + inspector.toString() + "\n ";

-            // errorMsg += "";

-            // List<Object> columns = ((StructObjectInspector)

-            // inspector).getStructFieldsDataAsList(cachedRowObject);

-            // for (Object column : columns) {

-            // errorMsg += column.toString() + " ";

-            // }

-            // errorMsg += "\n";

-            // Log.info(errorMsg);

-            // System.out.println(errorMsg);

-            // // result = new BooleanWritable(true);

-            // throw new IllegalStateException(errorMsg);

-            // }

+			// if (result == null) {

+			// result = evaluator.evaluate(cachedRowObject);

+			//

+			// // check if result is null

+			//

+			// String errorMsg = "serialize null object in  \n output " +

+			// outputInspector.toString() + " \n input "

+			// + inspector.toString() + "\n ";

+			// errorMsg += "";

+			// List<Object> columns = ((StructObjectInspector)

+			// inspector).getStructFieldsDataAsList(cachedRowObject);

+			// for (Object column : columns) {

+			// errorMsg += column.toString() + " ";

+			// }

+			// errorMsg += "\n";

+			// Log.info(errorMsg);

+			// System.out.println(errorMsg);

+			// // result = new BooleanWritable(true);

+			// throw new IllegalStateException(errorMsg);

+			// }

 

-            serializeResult(result);

-        } catch (HiveException e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e.getMessage());

-        } catch (IOException e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e.getMessage());

-        }

-    }

+			serializeResult(result);

+		} catch (HiveException e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e.getMessage());

+		} catch (IOException e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e.getMessage());

+		}

+	}

 

-    /**

-     * serialize the result

-     * 

-     * @param result

-     *            the evaluation result

-     * @throws IOException

-     * @throws AlgebricksException

-     */

-    private void serializeResult(Object result) throws IOException, AlgebricksException {

-        if (lazySer == null)

-            lazySer = new LazySerDe();

+	/**

+	 * serialize the result

+	 * 

+	 * @param result

+	 *            the evaluation result

+	 * @throws IOException

+	 * @throws AlgebricksException

+	 */

+	private void serializeResult(Object result) throws IOException,

+			AlgebricksException {

+		if (lazySer == null)

+			lazySer = new LazySerDe();

 

-        try {

-            BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, outputInspector);

-            dataOutput.write(outputWritable.getBytes(), 0, outputWritable.getLength());

-        } catch (SerDeException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+		try {

+			BytesWritable outputWritable = (BytesWritable) lazySer.serialize(

+					result, outputInspector);

+			dataOutput.write(outputWritable.getBytes(), 0,

+					outputWritable.getLength());

+		} catch (SerDeException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    /**

-     * bind the tuple reference to the cached row object

-     * 

-     * @param r

-     */

-    private void readIntoCache(IFrameTupleReference r) {

-        if (cachedRowObject == null)

-            cachedRowObject = (LazyObject<? extends ObjectInspector>) LazyFactory.createLazyObject(inspector);

-        cachedRowObject.init(r);

-    }

+	/**

+	 * bind the tuple reference to the cached row object

+	 * 

+	 * @param r

+	 */

+	private void readIntoCache(IFrameTupleReference r) {

+		if (cachedRowObject == null)

+			cachedRowObject = (LazyObject<? extends ObjectInspector>) LazyFactory

+					.createLazyObject(inspector);

+		cachedRowObject.init(r);

+	}

 

-    /**

-     * set a list of children of this evaluator

-     * 

-     * @param children

-     */

-    public void setChildren(List<ICopyEvaluator> children) {

-        this.children = children;

-    }

+	/**

+	 * set a list of children of this evaluator

+	 * 

+	 * @param children

+	 */

+	public void setChildren(List<ICopyEvaluator> children) {

+		this.children = children;

+	}

 

-    public void addChild(ICopyEvaluator child) {

-        if (children == null)

-            children = new ArrayList<ICopyEvaluator>();

-        children.add(child);

-    }

+	public void addChild(ICopyEvaluator child) {

+		if (children == null)

+			children = new ArrayList<ICopyEvaluator>();

+		children.add(child);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
index e500376..271b5e4 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregationFunctionEvaluator.java
@@ -24,201 +24,208 @@
 

 public class AggregationFunctionEvaluator implements ICopyAggregateFunction {

 

-    /**

-     * the mode of aggregation function

-     */

-    private GenericUDAFEvaluator.Mode mode;

+	/**

+	 * the mode of aggregation function

+	 */

+	private GenericUDAFEvaluator.Mode mode;

 

-    /**

-     * an array of evaluators

-     */

-    private ExprNodeEvaluator[] evaluators;

+	/**

+	 * an array of evaluators

+	 */

+	private ExprNodeEvaluator[] evaluators;

 

-    /**

-     * udaf evaluator partial

-     */

-    private GenericUDAFEvaluator udafPartial;

+	/**

+	 * udaf evaluator partial

+	 */

+	private GenericUDAFEvaluator udafPartial;

 

-    /**

-     * udaf evaluator complete

-     */

-    private GenericUDAFEvaluator udafComplete;

+	/**

+	 * udaf evaluator complete

+	 */

+	private GenericUDAFEvaluator udafComplete;

 

-    /**

-     * cached parameter objects

-     */

-    private Object[] cachedParameters;

+	/**

+	 * cached parameter objects

+	 */

+	private Object[] cachedParameters;

 

-    /**

-     * cached row objects

-     */

-    private LazyObject<? extends ObjectInspector> cachedRowObject;

+	/**

+	 * cached row objects

+	 */

+	private LazyObject<? extends ObjectInspector> cachedRowObject;

 

-    /**

-     * the output channel

-     */

-    private DataOutput out;

+	/**

+	 * the output channel

+	 */

+	private DataOutput out;

 

-    /**

-     * aggregation buffer

-     */

-    private AggregationBuffer aggBuffer;

+	/**

+	 * aggregation buffer

+	 */

+	private AggregationBuffer aggBuffer;

 

-    /**

-     * we only use lazy serde to do serialization

-     */

-    private SerDe lazySer;

+	/**

+	 * we only use lazy serde to do serialization

+	 */

+	private SerDe lazySer;

 

-    /**

-     * the output object inspector for this aggregation function

-     */

-    private ObjectInspector outputInspector;

+	/**

+	 * the output object inspector for this aggregation function

+	 */

+	private ObjectInspector outputInspector;

 

-    /**

-     * the output object inspector for this aggregation function

-     */

-    private ObjectInspector outputInspectorPartial;

+	/**

+	 * the output object inspector for this aggregation function

+	 */

+	private ObjectInspector outputInspectorPartial;

 

-    /**

-     * parameter inspectors

-     */

-    private ObjectInspector[] parameterInspectors;

+	/**

+	 * parameter inspectors

+	 */

+	private ObjectInspector[] parameterInspectors;

 

-    /**

-     * output make sure the aggregation functio has least object creation

-     * 

-     * @param desc

-     * @param oi

-     * @param output

-     */

-    public AggregationFunctionEvaluator(List<ExprNodeDesc> inputs, List<TypeInfo> inputTypes, String genericUDAFName,

-            GenericUDAFEvaluator.Mode aggMode, boolean distinct, ObjectInspector oi, DataOutput output,

-            ExprNodeEvaluator[] evals, ObjectInspector[] pInspectors, Object[] parameterCache, SerDe serde,

-            LazyObject<? extends ObjectInspector> row, GenericUDAFEvaluator udafunctionPartial,

-            GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi, ObjectInspector outputOiPartial) {

-        // shared object across threads

-        this.out = output;

-        this.mode = aggMode;

-        this.parameterInspectors = pInspectors;

+	/**

+	 * output make sure the aggregation functio has least object creation

+	 * 

+	 * @param desc

+	 * @param oi

+	 * @param output

+	 */

+	public AggregationFunctionEvaluator(List<ExprNodeDesc> inputs,

+			List<TypeInfo> inputTypes, String genericUDAFName,

+			GenericUDAFEvaluator.Mode aggMode, boolean distinct,

+			ObjectInspector oi, DataOutput output, ExprNodeEvaluator[] evals,

+			ObjectInspector[] pInspectors, Object[] parameterCache,

+			SerDe serde, LazyObject<? extends ObjectInspector> row,

+			GenericUDAFEvaluator udafunctionPartial,

+			GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi,

+			ObjectInspector outputOiPartial) {

+		// shared object across threads

+		this.out = output;

+		this.mode = aggMode;

+		this.parameterInspectors = pInspectors;

 

-        // thread local objects

-        this.evaluators = evals;

-        this.cachedParameters = parameterCache;

-        this.cachedRowObject = row;

-        this.lazySer = serde;

-        this.udafPartial = udafunctionPartial;

-        this.udafComplete = udafunctionComplete;

-        this.outputInspector = outputOi;

-        this.outputInspectorPartial = outputOiPartial;

-    }

+		// thread local objects

+		this.evaluators = evals;

+		this.cachedParameters = parameterCache;

+		this.cachedRowObject = row;

+		this.lazySer = serde;

+		this.udafPartial = udafunctionPartial;

+		this.udafComplete = udafunctionComplete;

+		this.outputInspector = outputOi;

+		this.outputInspectorPartial = outputOiPartial;

+	}

 

-    @Override

-    public void init() throws AlgebricksException {

-        try {

-            aggBuffer = udafPartial.getNewAggregationBuffer();

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	@Override

+	public void init() throws AlgebricksException {

+		try {

+			aggBuffer = udafPartial.getNewAggregationBuffer();

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    @Override

-    public void step(IFrameTupleReference tuple) throws AlgebricksException {

-        readIntoCache(tuple);

-        processRow();

-    }

+	@Override

+	public void step(IFrameTupleReference tuple) throws AlgebricksException {

+		readIntoCache(tuple);

+		processRow();

+	}

 

-    private void processRow() throws AlgebricksException {

-        try {

-            // get values by evaluating them

-            for (int i = 0; i < cachedParameters.length; i++) {

-                cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);

-            }

-            processAggregate();

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	private void processRow() throws AlgebricksException {

+		try {

+			// get values by evaluating them

+			for (int i = 0; i < cachedParameters.length; i++) {

+				cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);

+			}

+			processAggregate();

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    private void processAggregate() throws HiveException {

-        /**

-         * accumulate the aggregation function

-         */

-        switch (mode) {

-            case PARTIAL1:

-            case COMPLETE:

-                udafPartial.iterate(aggBuffer, cachedParameters);

-                break;

-            case PARTIAL2:

-            case FINAL:

-                if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {

-                    Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])

-                            .getPrimitiveWritableObject(cachedParameters[0]);

-                    udafPartial.merge(aggBuffer, parameter);

-                } else

-                    udafPartial.merge(aggBuffer, cachedParameters[0]);

-                break;

-            default:

-                break;

-        }

-    }

+	private void processAggregate() throws HiveException {

+		/**

+		 * accumulate the aggregation function

+		 */

+		switch (mode) {

+		case PARTIAL1:

+		case COMPLETE:

+			udafPartial.iterate(aggBuffer, cachedParameters);

+			break;

+		case PARTIAL2:

+		case FINAL:

+			if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {

+				Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])

+						.getPrimitiveWritableObject(cachedParameters[0]);

+				udafPartial.merge(aggBuffer, parameter);

+			} else

+				udafPartial.merge(aggBuffer, cachedParameters[0]);

+			break;

+		default:

+			break;

+		}

+	}

 

-    /**

-     * serialize the result

-     * 

-     * @param result

-     *            the evaluation result

-     * @throws IOException

-     * @throws AlgebricksException

-     */

-    private void serializeResult(Object result, ObjectInspector oi) throws IOException, AlgebricksException {

-        try {

-            BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, oi);

-            out.write(outputWritable.getBytes(), 0, outputWritable.getLength());

-        } catch (SerDeException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	/**

+	 * serialize the result

+	 * 

+	 * @param result

+	 *            the evaluation result

+	 * @throws IOException

+	 * @throws AlgebricksException

+	 */

+	private void serializeResult(Object result, ObjectInspector oi)

+			throws IOException, AlgebricksException {

+		try {

+			BytesWritable outputWritable = (BytesWritable) lazySer.serialize(

+					result, oi);

+			out.write(outputWritable.getBytes(), 0, outputWritable.getLength());

+		} catch (SerDeException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    /**

-     * bind the tuple reference to the cached row object

-     * 

-     * @param r

-     */

-    private void readIntoCache(IFrameTupleReference r) {

-        cachedRowObject.init(r);

-    }

+	/**

+	 * bind the tuple reference to the cached row object

+	 * 

+	 * @param r

+	 */

+	private void readIntoCache(IFrameTupleReference r) {

+		cachedRowObject.init(r);

+	}

 

-    @Override

-    public void finish() throws AlgebricksException {

-        // aggregator

-        try {

-            Object result = null;

-            result = udafPartial.terminatePartial(aggBuffer);

-            if (mode == GenericUDAFEvaluator.Mode.COMPLETE || mode == GenericUDAFEvaluator.Mode.FINAL) {

-                result = udafComplete.terminate(aggBuffer);

-                serializeResult(result, outputInspector);

-            } else {

-                serializeResult(result, outputInspectorPartial);

-            }

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        } catch (IOException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	@Override

+	public void finish() throws AlgebricksException {

+		// aggregator

+		try {

+			Object result = null;

+			result = udafPartial.terminatePartial(aggBuffer);

+			if (mode == GenericUDAFEvaluator.Mode.COMPLETE

+					|| mode == GenericUDAFEvaluator.Mode.FINAL) {

+				result = udafComplete.terminate(aggBuffer);

+				serializeResult(result, outputInspector);

+			} else {

+				serializeResult(result, outputInspectorPartial);

+			}

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		} catch (IOException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    @Override

-    public void finishPartial() throws AlgebricksException {

-        // aggregator.

-        try {

-            Object result = null;

-            // get aggregations

-            result = udafPartial.terminatePartial(aggBuffer);

-            serializeResult(result, outputInspectorPartial);

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        } catch (IOException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	@Override

+	public void finishPartial() throws AlgebricksException {

+		// aggregator.

+		try {

+			Object result = null;

+			// get aggregations

+			result = udafPartial.terminatePartial(aggBuffer);

+			serializeResult(result, outputInspectorPartial);

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		} catch (IOException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
index 1933253..032437b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/AggregatuibFunctionSerializableEvaluator.java
@@ -21,228 +21,239 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class AggregatuibFunctionSerializableEvaluator implements ICopySerializableAggregateFunction {
+public class AggregatuibFunctionSerializableEvaluator implements
+		ICopySerializableAggregateFunction {
 
-    /**
-     * the mode of aggregation function
-     */
-    private GenericUDAFEvaluator.Mode mode;
+	/**
+	 * the mode of aggregation function
+	 */
+	private GenericUDAFEvaluator.Mode mode;
 
-    /**
-     * an array of evaluators
-     */
-    private ExprNodeEvaluator[] evaluators;
+	/**
+	 * an array of evaluators
+	 */
+	private ExprNodeEvaluator[] evaluators;
 
-    /**
-     * udaf evaluator partial
-     */
-    private GenericUDAFEvaluator udafPartial;
+	/**
+	 * udaf evaluator partial
+	 */
+	private GenericUDAFEvaluator udafPartial;
 
-    /**
-     * udaf evaluator complete
-     */
-    private GenericUDAFEvaluator udafComplete;
+	/**
+	 * udaf evaluator complete
+	 */
+	private GenericUDAFEvaluator udafComplete;
 
-    /**
-     * cached parameter objects
-     */
-    private Object[] cachedParameters;
+	/**
+	 * cached parameter objects
+	 */
+	private Object[] cachedParameters;
 
-    /**
-     * cached row objects
-     */
-    private LazyObject<? extends ObjectInspector> cachedRowObject;
+	/**
+	 * cached row objects
+	 */
+	private LazyObject<? extends ObjectInspector> cachedRowObject;
 
-    /**
-     * aggregation buffer
-     */
-    private SerializableBuffer aggBuffer;
+	/**
+	 * aggregation buffer
+	 */
+	private SerializableBuffer aggBuffer;
 
-    /**
-     * we only use lazy serde to do serialization
-     */
-    private SerDe lazySer;
+	/**
+	 * we only use lazy serde to do serialization
+	 */
+	private SerDe lazySer;
 
-    /**
-     * the output object inspector for this aggregation function
-     */
-    private ObjectInspector outputInspector;
+	/**
+	 * the output object inspector for this aggregation function
+	 */
+	private ObjectInspector outputInspector;
 
-    /**
-     * the output object inspector for this aggregation function
-     */
-    private ObjectInspector outputInspectorPartial;
+	/**
+	 * the output object inspector for this aggregation function
+	 */
+	private ObjectInspector outputInspectorPartial;
 
-    /**
-     * parameter inspectors
-     */
-    private ObjectInspector[] parameterInspectors;
+	/**
+	 * parameter inspectors
+	 */
+	private ObjectInspector[] parameterInspectors;
 
-    /**
-     * output make sure the aggregation functio has least object creation
-     * 
-     * @param desc
-     * @param oi
-     * @param output
-     */
-    public AggregatuibFunctionSerializableEvaluator(List<ExprNodeDesc> inputs, List<TypeInfo> inputTypes,
-            String genericUDAFName, GenericUDAFEvaluator.Mode aggMode, boolean distinct, ObjectInspector oi,
-            ExprNodeEvaluator[] evals, ObjectInspector[] pInspectors, Object[] parameterCache, SerDe serde,
-            LazyObject<? extends ObjectInspector> row, GenericUDAFEvaluator udafunctionPartial,
-            GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi, ObjectInspector outputOiPartial)
-            throws AlgebricksException {
-        // shared object across threads
-        this.mode = aggMode;
-        this.parameterInspectors = pInspectors;
+	/**
+	 * output make sure the aggregation functio has least object creation
+	 * 
+	 * @param desc
+	 * @param oi
+	 * @param output
+	 */
+	public AggregatuibFunctionSerializableEvaluator(List<ExprNodeDesc> inputs,
+			List<TypeInfo> inputTypes, String genericUDAFName,
+			GenericUDAFEvaluator.Mode aggMode, boolean distinct,
+			ObjectInspector oi, ExprNodeEvaluator[] evals,
+			ObjectInspector[] pInspectors, Object[] parameterCache,
+			SerDe serde, LazyObject<? extends ObjectInspector> row,
+			GenericUDAFEvaluator udafunctionPartial,
+			GenericUDAFEvaluator udafunctionComplete, ObjectInspector outputOi,
+			ObjectInspector outputOiPartial) throws AlgebricksException {
+		// shared object across threads
+		this.mode = aggMode;
+		this.parameterInspectors = pInspectors;
 
-        // thread local objects
-        this.evaluators = evals;
-        this.cachedParameters = parameterCache;
-        this.cachedRowObject = row;
-        this.lazySer = serde;
-        this.udafPartial = udafunctionPartial;
-        this.udafComplete = udafunctionComplete;
-        this.outputInspector = outputOi;
-        this.outputInspectorPartial = outputOiPartial;
+		// thread local objects
+		this.evaluators = evals;
+		this.cachedParameters = parameterCache;
+		this.cachedRowObject = row;
+		this.lazySer = serde;
+		this.udafPartial = udafunctionPartial;
+		this.udafComplete = udafunctionComplete;
+		this.outputInspector = outputOi;
+		this.outputInspectorPartial = outputOiPartial;
 
-        try {
-            aggBuffer = (SerializableBuffer) udafPartial.getNewAggregationBuffer();
-        } catch (HiveException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+		try {
+			aggBuffer = (SerializableBuffer) udafPartial
+					.getNewAggregationBuffer();
+		} catch (HiveException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    @Override
-    public void init(DataOutput output) throws AlgebricksException {
-        try {
-            udafPartial.reset(aggBuffer);
-            outputAggBuffer(aggBuffer, output);
-        } catch (HiveException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	@Override
+	public void init(DataOutput output) throws AlgebricksException {
+		try {
+			udafPartial.reset(aggBuffer);
+			outputAggBuffer(aggBuffer, output);
+		} catch (HiveException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    @Override
-    public void step(IFrameTupleReference tuple, byte[] data, int start, int len) throws AlgebricksException {
-        deSerializeAggBuffer(aggBuffer, data, start, len);
-        readIntoCache(tuple);
-        processRow();
-        serializeAggBuffer(aggBuffer, data, start, len);
-    }
+	@Override
+	public void step(IFrameTupleReference tuple, byte[] data, int start, int len)
+			throws AlgebricksException {
+		deSerializeAggBuffer(aggBuffer, data, start, len);
+		readIntoCache(tuple);
+		processRow();
+		serializeAggBuffer(aggBuffer, data, start, len);
+	}
 
-    private void processRow() throws AlgebricksException {
-        try {
-            // get values by evaluating them
-            for (int i = 0; i < cachedParameters.length; i++) {
-                cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);
-            }
-            processAggregate();
-        } catch (HiveException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	private void processRow() throws AlgebricksException {
+		try {
+			// get values by evaluating them
+			for (int i = 0; i < cachedParameters.length; i++) {
+				cachedParameters[i] = evaluators[i].evaluate(cachedRowObject);
+			}
+			processAggregate();
+		} catch (HiveException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    private void processAggregate() throws HiveException {
-        /**
-         * accumulate the aggregation function
-         */
-        switch (mode) {
-            case PARTIAL1:
-            case COMPLETE:
-                udafPartial.iterate(aggBuffer, cachedParameters);
-                break;
-            case PARTIAL2:
-            case FINAL:
-                if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {
-                    Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])
-                            .getPrimitiveWritableObject(cachedParameters[0]);
-                    udafPartial.merge(aggBuffer, parameter);
-                } else
-                    udafPartial.merge(aggBuffer, cachedParameters[0]);
-                break;
-            default:
-                break;
-        }
-    }
+	private void processAggregate() throws HiveException {
+		/**
+		 * accumulate the aggregation function
+		 */
+		switch (mode) {
+		case PARTIAL1:
+		case COMPLETE:
+			udafPartial.iterate(aggBuffer, cachedParameters);
+			break;
+		case PARTIAL2:
+		case FINAL:
+			if (udafPartial instanceof GenericUDAFCount.GenericUDAFCountEvaluator) {
+				Object parameter = ((PrimitiveObjectInspector) parameterInspectors[0])
+						.getPrimitiveWritableObject(cachedParameters[0]);
+				udafPartial.merge(aggBuffer, parameter);
+			} else
+				udafPartial.merge(aggBuffer, cachedParameters[0]);
+			break;
+		default:
+			break;
+		}
+	}
 
-    /**
-     * serialize the result
-     * 
-     * @param result
-     *            the evaluation result
-     * @throws IOException
-     * @throws AlgebricksException
-     */
-    private void serializeResult(Object result, ObjectInspector oi, DataOutput out) throws IOException,
-            AlgebricksException {
-        try {
-            BytesWritable outputWritable = (BytesWritable) lazySer.serialize(result, oi);
-            out.write(outputWritable.getBytes(), 0, outputWritable.getLength());
-        } catch (SerDeException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	/**
+	 * serialize the result
+	 * 
+	 * @param result
+	 *            the evaluation result
+	 * @throws IOException
+	 * @throws AlgebricksException
+	 */
+	private void serializeResult(Object result, ObjectInspector oi,
+			DataOutput out) throws IOException, AlgebricksException {
+		try {
+			BytesWritable outputWritable = (BytesWritable) lazySer.serialize(
+					result, oi);
+			out.write(outputWritable.getBytes(), 0, outputWritable.getLength());
+		} catch (SerDeException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    /**
-     * bind the tuple reference to the cached row object
-     * 
-     * @param r
-     */
-    private void readIntoCache(IFrameTupleReference r) {
-        cachedRowObject.init(r);
-    }
+	/**
+	 * bind the tuple reference to the cached row object
+	 * 
+	 * @param r
+	 */
+	private void readIntoCache(IFrameTupleReference r) {
+		cachedRowObject.init(r);
+	}
 
-    @Override
-    public void finish(byte[] data, int start, int len, DataOutput output) throws AlgebricksException {
-        deSerializeAggBuffer(aggBuffer, data, start, len);
-        // aggregator
-        try {
-            Object result = null;
-            result = udafPartial.terminatePartial(aggBuffer);
-            if (mode == GenericUDAFEvaluator.Mode.COMPLETE || mode == GenericUDAFEvaluator.Mode.FINAL) {
-                result = udafComplete.terminate(aggBuffer);
-                serializeResult(result, outputInspector, output);
-            } else {
-                serializeResult(result, outputInspectorPartial, output);
-            }
-        } catch (HiveException e) {
-            throw new AlgebricksException(e);
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	@Override
+	public void finish(byte[] data, int start, int len, DataOutput output)
+			throws AlgebricksException {
+		deSerializeAggBuffer(aggBuffer, data, start, len);
+		// aggregator
+		try {
+			Object result = null;
+			result = udafPartial.terminatePartial(aggBuffer);
+			if (mode == GenericUDAFEvaluator.Mode.COMPLETE
+					|| mode == GenericUDAFEvaluator.Mode.FINAL) {
+				result = udafComplete.terminate(aggBuffer);
+				serializeResult(result, outputInspector, output);
+			} else {
+				serializeResult(result, outputInspectorPartial, output);
+			}
+		} catch (HiveException e) {
+			throw new AlgebricksException(e);
+		} catch (IOException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    @Override
-    public void finishPartial(byte[] data, int start, int len, DataOutput output) throws AlgebricksException {
-        deSerializeAggBuffer(aggBuffer, data, start, len);
-        // aggregator.
-        try {
-            Object result = null;
-            // get aggregations
-            result = udafPartial.terminatePartial(aggBuffer);
-            serializeResult(result, outputInspectorPartial, output);
-        } catch (HiveException e) {
-            throw new AlgebricksException(e);
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	@Override
+	public void finishPartial(byte[] data, int start, int len, DataOutput output)
+			throws AlgebricksException {
+		deSerializeAggBuffer(aggBuffer, data, start, len);
+		// aggregator.
+		try {
+			Object result = null;
+			// get aggregations
+			result = udafPartial.terminatePartial(aggBuffer);
+			serializeResult(result, outputInspectorPartial, output);
+		} catch (HiveException e) {
+			throw new AlgebricksException(e);
+		} catch (IOException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    private void serializeAggBuffer(SerializableBuffer buffer, byte[] data, int start, int len)
-            throws AlgebricksException {
-        buffer.serializeAggBuffer(data, start, len);
-    }
+	private void serializeAggBuffer(SerializableBuffer buffer, byte[] data,
+			int start, int len) throws AlgebricksException {
+		buffer.serializeAggBuffer(data, start, len);
+	}
 
-    private void deSerializeAggBuffer(SerializableBuffer buffer, byte[] data, int start, int len)
-            throws AlgebricksException {
-        buffer.deSerializeAggBuffer(data, start, len);
-    }
+	private void deSerializeAggBuffer(SerializableBuffer buffer, byte[] data,
+			int start, int len) throws AlgebricksException {
+		buffer.deSerializeAggBuffer(data, start, len);
+	}
 
-    private void outputAggBuffer(SerializableBuffer buffer, DataOutput out) throws AlgebricksException {
-        try {
-            buffer.serializeAggBuffer(out);
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	private void outputAggBuffer(SerializableBuffer buffer, DataOutput out)
+			throws AlgebricksException {
+		try {
+			buffer.serializeAggBuffer(out);
+		} catch (IOException e) {
+			throw new AlgebricksException(e);
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java
index 96065e5..d73be93 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/BufferSerDeUtil.java
@@ -2,66 +2,72 @@
 
 public class BufferSerDeUtil {
 
-    public static double getDouble(byte[] bytes, int offset) {
-        return Double.longBitsToDouble(getLong(bytes, offset));
-    }
+	public static double getDouble(byte[] bytes, int offset) {
+		return Double.longBitsToDouble(getLong(bytes, offset));
+	}
 
-    public static float getFloat(byte[] bytes, int offset) {
-        return Float.intBitsToFloat(getInt(bytes, offset));
-    }
+	public static float getFloat(byte[] bytes, int offset) {
+		return Float.intBitsToFloat(getInt(bytes, offset));
+	}
 
-    public static boolean getBoolean(byte[] bytes, int offset) {
-        if (bytes[offset] == 0)
-            return false;
-        else
-            return true;
-    }
+	public static boolean getBoolean(byte[] bytes, int offset) {
+		if (bytes[offset] == 0)
+			return false;
+		else
+			return true;
+	}
 
-    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);
-    }
+	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);
+	}
 
-    public static long getLong(byte[] bytes, int offset) {
-        return (((long) (bytes[offset] & 0xff)) << 56) + (((long) (bytes[offset + 1] & 0xff)) << 48)
-                + (((long) (bytes[offset + 2] & 0xff)) << 40) + (((long) (bytes[offset + 3] & 0xff)) << 32)
-                + (((long) (bytes[offset + 4] & 0xff)) << 24) + (((long) (bytes[offset + 5] & 0xff)) << 16)
-                + (((long) (bytes[offset + 6] & 0xff)) << 8) + (((long) (bytes[offset + 7] & 0xff)) << 0);
-    }
+	public static long getLong(byte[] bytes, int offset) {
+		return (((long) (bytes[offset] & 0xff)) << 56)
+				+ (((long) (bytes[offset + 1] & 0xff)) << 48)
+				+ (((long) (bytes[offset + 2] & 0xff)) << 40)
+				+ (((long) (bytes[offset + 3] & 0xff)) << 32)
+				+ (((long) (bytes[offset + 4] & 0xff)) << 24)
+				+ (((long) (bytes[offset + 5] & 0xff)) << 16)
+				+ (((long) (bytes[offset + 6] & 0xff)) << 8)
+				+ (((long) (bytes[offset + 7] & 0xff)) << 0);
+	}
 
-    public static void writeBoolean(boolean value, byte[] bytes, int offset) {
-        if (value)
-            bytes[offset] = (byte) 1;
-        else
-            bytes[offset] = (byte) 0;
-    }
+	public static void writeBoolean(boolean value, byte[] bytes, int offset) {
+		if (value)
+			bytes[offset] = (byte) 1;
+		else
+			bytes[offset] = (byte) 0;
+	}
 
-    public static void writeInt(int value, byte[] bytes, int offset) {
-        bytes[offset++] = (byte) (value >> 24);
-        bytes[offset++] = (byte) (value >> 16);
-        bytes[offset++] = (byte) (value >> 8);
-        bytes[offset++] = (byte) (value);
-    }
+	public static void writeInt(int value, byte[] bytes, int offset) {
+		bytes[offset++] = (byte) (value >> 24);
+		bytes[offset++] = (byte) (value >> 16);
+		bytes[offset++] = (byte) (value >> 8);
+		bytes[offset++] = (byte) (value);
+	}
 
-    public static void writeLong(long value, byte[] bytes, int offset) {
-        bytes[offset++] = (byte) (value >> 56);
-        bytes[offset++] = (byte) (value >> 48);
-        bytes[offset++] = (byte) (value >> 40);
-        bytes[offset++] = (byte) (value >> 32);
-        bytes[offset++] = (byte) (value >> 24);
-        bytes[offset++] = (byte) (value >> 16);
-        bytes[offset++] = (byte) (value >> 8);
-        bytes[offset++] = (byte) (value);
-    }
+	public static void writeLong(long value, byte[] bytes, int offset) {
+		bytes[offset++] = (byte) (value >> 56);
+		bytes[offset++] = (byte) (value >> 48);
+		bytes[offset++] = (byte) (value >> 40);
+		bytes[offset++] = (byte) (value >> 32);
+		bytes[offset++] = (byte) (value >> 24);
+		bytes[offset++] = (byte) (value >> 16);
+		bytes[offset++] = (byte) (value >> 8);
+		bytes[offset++] = (byte) (value);
+	}
 
-    public static void writeDouble(double value, byte[] bytes, int offset) {
-        long lValue = Double.doubleToLongBits(value);
-        writeLong(lValue, bytes, offset);
-    }
+	public static void writeDouble(double value, byte[] bytes, int offset) {
+		long lValue = Double.doubleToLongBits(value);
+		writeLong(lValue, bytes, offset);
+	}
 
-    public static void writeFloat(float value, byte[] bytes, int offset) {
-        int iValue = Float.floatToIntBits(value);
-        writeInt(iValue, bytes, offset);
-    }
+	public static void writeFloat(float value, byte[] bytes, int offset) {
+		int iValue = Float.floatToIntBits(value);
+		writeInt(iValue, bytes, offset);
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java
index 5647f6a..3296e19 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ColumnExpressionEvaluator.java
@@ -9,9 +9,10 @@
 

 public class ColumnExpressionEvaluator extends AbstractExpressionEvaluator {

 

-    public ColumnExpressionEvaluator(ExprNodeColumnDesc expr, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        super(new ExprNodeColumnEvaluator(expr), oi, output);

-    }

+	public ColumnExpressionEvaluator(ExprNodeColumnDesc expr,

+			ObjectInspector oi, IDataOutputProvider output)

+			throws AlgebricksException {

+		super(new ExprNodeColumnEvaluator(expr), oi, output);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java
index d8796ea..62928e6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ConstantExpressionEvaluator.java
@@ -9,8 +9,9 @@
 

 public class ConstantExpressionEvaluator extends AbstractExpressionEvaluator {

 

-    public ConstantExpressionEvaluator(ExprNodeConstantDesc expr, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        super(new ExprNodeConstantEvaluator(expr), oi, output);

-    }

+	public ConstantExpressionEvaluator(ExprNodeConstantDesc expr,

+			ObjectInspector oi, IDataOutputProvider output)

+			throws AlgebricksException {

+		super(new ExprNodeConstantEvaluator(expr), oi, output);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java
index c40ef73..2180910 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/ExpressionTranslator.java
@@ -41,173 +41,193 @@
 

 public class ExpressionTranslator {

 

-    public static Object getHiveExpression(ILogicalExpression expr, IVariableTypeEnvironment env) throws Exception {

-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {

-            /**

-             * function expression

-             */

-            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;

-            IFunctionInfo funcInfo = funcExpr.getFunctionInfo();

-            FunctionIdentifier fid = funcInfo.getFunctionIdentifier();

+	public static Object getHiveExpression(ILogicalExpression expr,

+			IVariableTypeEnvironment env) throws Exception {

+		if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {

+			/**

+			 * function expression

+			 */

+			AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;

+			IFunctionInfo funcInfo = funcExpr.getFunctionInfo();

+			FunctionIdentifier fid = funcInfo.getFunctionIdentifier();

 

-            if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

-                Object info = ((HiveFunctionInfo) funcInfo).getInfo();

-                ExprNodeFieldDesc desc = (ExprNodeFieldDesc) info;

-                return new ExprNodeFieldDesc(desc.getTypeInfo(), desc.getDesc(), desc.getFieldName(), desc.getIsList());

-            }

+			if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

+				Object info = ((HiveFunctionInfo) funcInfo).getInfo();

+				ExprNodeFieldDesc desc = (ExprNodeFieldDesc) info;

+				return new ExprNodeFieldDesc(desc.getTypeInfo(),

+						desc.getDesc(), desc.getFieldName(), desc.getIsList());

+			}

 

-            if (fid.getName().equals(ExpressionConstant.NULL)) {

-                return new ExprNodeNullDesc();

-            }

+			if (fid.getName().equals(ExpressionConstant.NULL)) {

+				return new ExprNodeNullDesc();

+			}

 

-            /**

-             * argument expressions: translate argument expressions recursively

-             * first, this logic is shared in scalar, aggregation and unnesting

-             * function

-             */

-            List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();

-            List<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();

-            for (Mutable<ILogicalExpression> argument : arguments) {

-                /**

-                 * parameters could not be aggregate function desc

-                 */

-                ExprNodeDesc parameter = (ExprNodeDesc) getHiveExpression(argument.getValue(), env);

-                parameters.add(parameter);

-            }

+			/**

+			 * argument expressions: translate argument expressions recursively

+			 * first, this logic is shared in scalar, aggregation and unnesting

+			 * function

+			 */

+			List<Mutable<ILogicalExpression>> arguments = funcExpr

+					.getArguments();

+			List<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();

+			for (Mutable<ILogicalExpression> argument : arguments) {

+				/**

+				 * parameters could not be aggregate function desc

+				 */

+				ExprNodeDesc parameter = (ExprNodeDesc) getHiveExpression(

+						argument.getValue(), env);

+				parameters.add(parameter);

+			}

 

-            /**

-             * get expression

-             */

-            if (funcExpr instanceof ScalarFunctionCallExpression) {

-                String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE.getHiveFunctionName(fid);

-                GenericUDF udf;

-                if (udfName != null) {

-                    /**

-                     * get corresponding function info for built-in functions

-                     */

-                    FunctionInfo fInfo = FunctionRegistry.getFunctionInfo(udfName);

-                    udf = fInfo.getGenericUDF();

+			/**

+			 * get expression

+			 */

+			if (funcExpr instanceof ScalarFunctionCallExpression) {

+				String udfName = HiveAlgebricksBuiltInFunctionMap.INSTANCE

+						.getHiveFunctionName(fid);

+				GenericUDF udf;

+				if (udfName != null) {

+					/**

+					 * get corresponding function info for built-in functions

+					 */

+					FunctionInfo fInfo = FunctionRegistry

+							.getFunctionInfo(udfName);

+					udf = fInfo.getGenericUDF();

 

-                    int inputSize = parameters.size();

-                    List<ExprNodeDesc> currentDescs = new ArrayList<ExprNodeDesc>();

+					int inputSize = parameters.size();

+					List<ExprNodeDesc> currentDescs = new ArrayList<ExprNodeDesc>();

 

-                    // generate expression tree if necessary

-                    while (inputSize > 2) {

-                        int pairs = inputSize / 2;

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

-                            List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>();

-                            descs.add(parameters.get(2 * i));

-                            descs.add(parameters.get(2 * i + 1));

-                            ExprNodeDesc desc = ExprNodeGenericFuncDesc.newInstance(udf, descs);

-                            currentDescs.add(desc);

-                        }

+					// generate expression tree if necessary

+					while (inputSize > 2) {

+						int pairs = inputSize / 2;

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

+							List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>();

+							descs.add(parameters.get(2 * i));

+							descs.add(parameters.get(2 * i + 1));

+							ExprNodeDesc desc = ExprNodeGenericFuncDesc

+									.newInstance(udf, descs);

+							currentDescs.add(desc);

+						}

 

-                        if (inputSize % 2 != 0) {

-                            // List<ExprNodeDesc> descs = new

-                            // ArrayList<ExprNodeDesc>();

-                            // ExprNodeDesc lastExpr =

-                            // currentDescs.remove(currentDescs.size() - 1);

-                            // descs.add(lastExpr);

-                            currentDescs.add(parameters.get(inputSize - 1));

-                            // ExprNodeDesc desc =

-                            // ExprNodeGenericFuncDesc.newInstance(udf, descs);

-                            // currentDescs.add(desc);

-                        }

-                        inputSize = currentDescs.size();

-                        parameters.clear();

-                        parameters.addAll(currentDescs);

-                        currentDescs.clear();

-                    }

+						if (inputSize % 2 != 0) {

+							// List<ExprNodeDesc> descs = new

+							// ArrayList<ExprNodeDesc>();

+							// ExprNodeDesc lastExpr =

+							// currentDescs.remove(currentDescs.size() - 1);

+							// descs.add(lastExpr);

+							currentDescs.add(parameters.get(inputSize - 1));

+							// ExprNodeDesc desc =

+							// ExprNodeGenericFuncDesc.newInstance(udf, descs);

+							// currentDescs.add(desc);

+						}

+						inputSize = currentDescs.size();

+						parameters.clear();

+						parameters.addAll(currentDescs);

+						currentDescs.clear();

+					}

 

-                } else {

-                    Object secondInfo = ((HiveFunctionInfo) funcInfo).getInfo();

-                    if (secondInfo != null) {

+				} else {

+					Object secondInfo = ((HiveFunctionInfo) funcInfo).getInfo();

+					if (secondInfo != null) {

 

-                        /**

-                         * for GenericUDFBridge: we should not call get type of

-                         * this hive expression, because parameters may have

-                         * been changed!

-                         */

-                        ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) ((HiveFunctionInfo) funcInfo)

-                                .getInfo();

-                        udf = hiveExpr.getGenericUDF();

-                    } else {

-                        /**

-                         * for other generic UDF

-                         */

-                        Class<?> udfClass;

-                        try {

-                            udfClass = Class.forName(fid.getName());

-                            udf = (GenericUDF) udfClass.newInstance();

-                        } catch (Exception e) {

-                            e.printStackTrace();

-                            throw new AlgebricksException(e.getMessage());

-                        }

-                    }

-                }

-                /**

-                 * get hive generic function expression

-                 */

-                ExprNodeDesc desc = ExprNodeGenericFuncDesc.newInstance(udf, parameters);

-                return desc;

-            } else if (funcExpr instanceof AggregateFunctionCallExpression) {

-                /**

-                 * hive aggregation info

-                 */

-                AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo())

-                        .getInfo();

-                /**

-                 * set parameters

-                 */

-                aggregateDesc.setParameters((ArrayList<ExprNodeDesc>) parameters);

+						/**

+						 * for GenericUDFBridge: we should not call get type of

+						 * this hive expression, because parameters may have

+						 * been changed!

+						 */

+						ExprNodeGenericFuncDesc hiveExpr = (ExprNodeGenericFuncDesc) ((HiveFunctionInfo) funcInfo)

+								.getInfo();

+						udf = hiveExpr.getGenericUDF();

+					} else {

+						/**

+						 * for other generic UDF

+						 */

+						Class<?> udfClass;

+						try {

+							udfClass = Class.forName(fid.getName());

+							udf = (GenericUDF) udfClass.newInstance();

+						} catch (Exception e) {

+							e.printStackTrace();

+							throw new AlgebricksException(e.getMessage());

+						}

+					}

+				}

+				/**

+				 * get hive generic function expression

+				 */

+				ExprNodeDesc desc = ExprNodeGenericFuncDesc.newInstance(udf,

+						parameters);

+				return desc;

+			} else if (funcExpr instanceof AggregateFunctionCallExpression) {

+				/**

+				 * hive aggregation info

+				 */

+				AggregationDesc aggregateDesc = (AggregationDesc) ((HiveFunctionInfo) funcExpr

+						.getFunctionInfo()).getInfo();

+				/**

+				 * set parameters

+				 */

+				aggregateDesc

+						.setParameters((ArrayList<ExprNodeDesc>) parameters);

 

-                List<TypeInfo> originalParameterTypeInfos = new ArrayList<TypeInfo>();

-                for (ExprNodeDesc parameter : parameters) {

-                    if (parameter.getTypeInfo() instanceof StructTypeInfo) {

-                        originalParameterTypeInfos.add(TypeInfoFactory.doubleTypeInfo);

-                    } else

-                        originalParameterTypeInfos.add(parameter.getTypeInfo());

-                }

+				List<TypeInfo> originalParameterTypeInfos = new ArrayList<TypeInfo>();

+				for (ExprNodeDesc parameter : parameters) {

+					if (parameter.getTypeInfo() instanceof StructTypeInfo) {

+						originalParameterTypeInfos

+								.add(TypeInfoFactory.doubleTypeInfo);

+					} else

+						originalParameterTypeInfos.add(parameter.getTypeInfo());

+				}

 

-                GenericUDAFEvaluator eval = FunctionRegistry.getGenericUDAFEvaluator(

-                        aggregateDesc.getGenericUDAFName(), originalParameterTypeInfos, aggregateDesc.getDistinct(),

-                        false);

+				GenericUDAFEvaluator eval = FunctionRegistry

+						.getGenericUDAFEvaluator(

+								aggregateDesc.getGenericUDAFName(),

+								originalParameterTypeInfos,

+								aggregateDesc.getDistinct(), false);

 

-                AggregationDesc newAggregateDesc = new AggregationDesc(aggregateDesc.getGenericUDAFName(), eval,

-                        aggregateDesc.getParameters(), aggregateDesc.getDistinct(), aggregateDesc.getMode());

-                return newAggregateDesc;

-            } else if (funcExpr instanceof UnnestingFunctionCallExpression) {

-                /**

-                 * type inference for UDTF function

-                 */

-                UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();

-                String funcName = hiveDesc.getUDTFName();

-                FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName);

-                GenericUDTF udtf = fi.getGenericUDTF();

-                UDTFDesc desc = new UDTFDesc(udtf);

-                return desc;

-            } else {

-                throw new IllegalStateException("unrecognized function expression " + expr.getClass().getName());

-            }

-        } else if ((expr.getExpressionTag() == LogicalExpressionTag.VARIABLE)) {

-            /**

-             * get type for variable in the environment

-             */

-            VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;

-            LogicalVariable var = varExpr.getVariableReference();

-            TypeInfo typeInfo = (TypeInfo) env.getVarType(var);

-            ExprNodeDesc desc = new ExprNodeColumnDesc(typeInfo, var.toString(), "", false);

-            return desc;

-        } else if ((expr.getExpressionTag() == LogicalExpressionTag.CONSTANT)) {

-            /**

-             * get expression for constant in the environment

-             */

-            ConstantExpression varExpr = (ConstantExpression) expr;

-            Object value = ((HivesterixConstantValue) varExpr.getValue()).getObject();

-            ExprNodeDesc desc = new ExprNodeConstantDesc(value);

-            return desc;

-        } else {

-            throw new IllegalStateException("illegal expressions " + expr.getClass().getName());

-        }

-    }

+				AggregationDesc newAggregateDesc = new AggregationDesc(

+						aggregateDesc.getGenericUDAFName(), eval,

+						aggregateDesc.getParameters(),

+						aggregateDesc.getDistinct(), aggregateDesc.getMode());

+				return newAggregateDesc;

+			} else if (funcExpr instanceof UnnestingFunctionCallExpression) {

+				/**

+				 * type inference for UDTF function

+				 */

+				UDTFDesc hiveDesc = (UDTFDesc) ((HiveFunctionInfo) funcExpr

+						.getFunctionInfo()).getInfo();

+				String funcName = hiveDesc.getUDTFName();

+				FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName);

+				GenericUDTF udtf = fi.getGenericUDTF();

+				UDTFDesc desc = new UDTFDesc(udtf);

+				return desc;

+			} else {

+				throw new IllegalStateException(

+						"unrecognized function expression "

+								+ expr.getClass().getName());

+			}

+		} else if ((expr.getExpressionTag() == LogicalExpressionTag.VARIABLE)) {

+			/**

+			 * get type for variable in the environment

+			 */

+			VariableReferenceExpression varExpr = (VariableReferenceExpression) expr;

+			LogicalVariable var = varExpr.getVariableReference();

+			TypeInfo typeInfo = (TypeInfo) env.getVarType(var);

+			ExprNodeDesc desc = new ExprNodeColumnDesc(typeInfo,

+					var.toString(), "", false);

+			return desc;

+		} else if ((expr.getExpressionTag() == LogicalExpressionTag.CONSTANT)) {

+			/**

+			 * get expression for constant in the environment

+			 */

+			ConstantExpression varExpr = (ConstantExpression) expr;

+			Object value = ((HivesterixConstantValue) varExpr.getValue())

+					.getObject();

+			ExprNodeDesc desc = new ExprNodeConstantDesc(value);

+			return desc;

+		} else {

+			throw new IllegalStateException("illegal expressions "

+					+ expr.getClass().getName());

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java
index 35560b6..5f6a5dc 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FieldExpressionEvaluator.java
@@ -9,9 +9,9 @@
 

 public class FieldExpressionEvaluator extends AbstractExpressionEvaluator {

 

-    public FieldExpressionEvaluator(ExprNodeFieldDesc expr, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        super(new ExprNodeFieldEvaluator(expr), oi, output);

-    }

+	public FieldExpressionEvaluator(ExprNodeFieldDesc expr, ObjectInspector oi,

+			IDataOutputProvider output) throws AlgebricksException {

+		super(new ExprNodeFieldEvaluator(expr), oi, output);

+	}

 

 }
\ No newline at end of file
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java
index 7ffec7a..c3f3c93 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/FunctionExpressionEvaluator.java
@@ -9,9 +9,10 @@
 

 public class FunctionExpressionEvaluator extends AbstractExpressionEvaluator {

 

-    public FunctionExpressionEvaluator(ExprNodeGenericFuncDesc expr, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        super(new ExprNodeGenericFuncEvaluator(expr), oi, output);

-    }

+	public FunctionExpressionEvaluator(ExprNodeGenericFuncDesc expr,

+			ObjectInspector oi, IDataOutputProvider output)

+			throws AlgebricksException {

+		super(new ExprNodeGenericFuncEvaluator(expr), oi, output);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java
index ca60385..cbe5561 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/NullExpressionEvaluator.java
@@ -9,8 +9,8 @@
 

 public class NullExpressionEvaluator extends AbstractExpressionEvaluator {

 

-    public NullExpressionEvaluator(ExprNodeNullDesc expr, ObjectInspector oi, IDataOutputProvider output)

-            throws AlgebricksException {

-        super(new ExprNodeNullEvaluator(expr), oi, output);

-    }

+	public NullExpressionEvaluator(ExprNodeNullDesc expr, ObjectInspector oi,

+			IDataOutputProvider output) throws AlgebricksException {

+		super(new ExprNodeNullEvaluator(expr), oi, output);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java
index 676989e..328b384 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/SerializableBuffer.java
@@ -7,10 +7,10 @@
 
 public interface SerializableBuffer extends AggregationBuffer {
 
-    public void deSerializeAggBuffer(byte[] data, int start, int len);
+	public void deSerializeAggBuffer(byte[] data, int start, int len);
 
-    public void serializeAggBuffer(byte[] data, int start, int len);
+	public void serializeAggBuffer(byte[] data, int start, int len);
 
-    public void serializeAggBuffer(DataOutput output) throws IOException;
+	public void serializeAggBuffer(DataOutput output) throws IOException;
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
index 284d3d2..de0141b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/evaluator/UDTFFunctionEvaluator.java
@@ -23,121 +23,125 @@
 

 public class UDTFFunctionEvaluator implements ICopyUnnestingFunction, Collector {

 

-    /**

-     * udtf function

-     */

-    private UDTFDesc func;

+	/**

+	 * udtf function

+	 */

+	private UDTFDesc func;

 

-    /**

-     * input object inspector

-     */

-    private ObjectInspector inputInspector;

+	/**

+	 * input object inspector

+	 */

+	private ObjectInspector inputInspector;

 

-    /**

-     * output object inspector

-     */

-    private ObjectInspector outputInspector;

+	/**

+	 * output object inspector

+	 */

+	private ObjectInspector outputInspector;

 

-    /**

-     * object inspector for udtf

-     */

-    private ObjectInspector[] udtfInputOIs;

+	/**

+	 * object inspector for udtf

+	 */

+	private ObjectInspector[] udtfInputOIs;

 

-    /**

-     * generic udtf

-     */

-    private GenericUDTF udtf;

+	/**

+	 * generic udtf

+	 */

+	private GenericUDTF udtf;

 

-    /**

-     * data output

-     */

-    private DataOutput out;

+	/**

+	 * data output

+	 */

+	private DataOutput out;

 

-    /**

-     * the input row object

-     */

-    private LazyColumnar cachedRowObject;

+	/**

+	 * the input row object

+	 */

+	private LazyColumnar cachedRowObject;

 

-    /**

-     * cached row object (input)

-     */

-    private Object[] cachedInputObjects;

+	/**

+	 * cached row object (input)

+	 */

+	private Object[] cachedInputObjects;

 

-    /**

-     * serialization/deserialization

-     */

-    private SerDe lazySerDe;

+	/**

+	 * serialization/deserialization

+	 */

+	private SerDe lazySerDe;

 

-    /**

-     * columns feed into UDTF

-     */

-    private int[] columns;

+	/**

+	 * columns feed into UDTF

+	 */

+	private int[] columns;

 

-    public UDTFFunctionEvaluator(UDTFDesc desc, Schema schema, int[] cols, DataOutput output) {

-        this.func = desc;

-        this.inputInspector = schema.toObjectInspector();

-        udtf = func.getGenericUDTF();

-        out = output;

-        columns = cols;

-    }

+	public UDTFFunctionEvaluator(UDTFDesc desc, Schema schema, int[] cols,

+			DataOutput output) {

+		this.func = desc;

+		this.inputInspector = schema.toObjectInspector();

+		udtf = func.getGenericUDTF();

+		out = output;

+		columns = cols;

+	}

 

-    @Override

-    public void init(IFrameTupleReference tuple) throws AlgebricksException {

-        cachedInputObjects = new LazyObject[columns.length];

-        try {

-            cachedRowObject = (LazyColumnar) LazyFactory.createLazyObject(inputInspector);

-            outputInspector = udtf.initialize(udtfInputOIs);

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        }

-        udtf.setCollector(this);

-        lazySerDe = new LazySerDe();

-        readIntoCache(tuple);

-    }

+	@Override

+	public void init(IFrameTupleReference tuple) throws AlgebricksException {

+		cachedInputObjects = new LazyObject[columns.length];

+		try {

+			cachedRowObject = (LazyColumnar) LazyFactory

+					.createLazyObject(inputInspector);

+			outputInspector = udtf.initialize(udtfInputOIs);

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		}

+		udtf.setCollector(this);

+		lazySerDe = new LazySerDe();

+		readIntoCache(tuple);

+	}

 

-    @Override

-    public boolean step() throws AlgebricksException {

-        try {

-            udtf.process(cachedInputObjects);

-            return true;

-        } catch (HiveException e) {

-            throw new AlgebricksException(e);

-        }

-    }

+	@Override

+	public boolean step() throws AlgebricksException {

+		try {

+			udtf.process(cachedInputObjects);

+			return true;

+		} catch (HiveException e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    /**

-     * bind the tuple reference to the cached row object

-     * 

-     * @param r

-     */

-    private void readIntoCache(IFrameTupleReference r) {

-        cachedRowObject.init(r);

-        for (int i = 0; i < cachedInputObjects.length; i++) {

-            cachedInputObjects[i] = cachedRowObject.getField(columns[i]);

-        }

-    }

+	/**

+	 * bind the tuple reference to the cached row object

+	 * 

+	 * @param r

+	 */

+	private void readIntoCache(IFrameTupleReference r) {

+		cachedRowObject.init(r);

+		for (int i = 0; i < cachedInputObjects.length; i++) {

+			cachedInputObjects[i] = cachedRowObject.getField(columns[i]);

+		}

+	}

 

-    /**

-     * serialize the result

-     * 

-     * @param result

-     *            the evaluation result

-     * @throws IOException

-     * @throws AlgebricksException

-     */

-    private void serializeResult(Object result) throws SerDeException, IOException {

-        BytesWritable outputWritable = (BytesWritable) lazySerDe.serialize(result, outputInspector);

-        out.write(outputWritable.getBytes(), 0, outputWritable.getLength());

-    }

+	/**

+	 * serialize the result

+	 * 

+	 * @param result

+	 *            the evaluation result

+	 * @throws IOException

+	 * @throws AlgebricksException

+	 */

+	private void serializeResult(Object result) throws SerDeException,

+			IOException {

+		BytesWritable outputWritable = (BytesWritable) lazySerDe.serialize(

+				result, outputInspector);

+		out.write(outputWritable.getBytes(), 0, outputWritable.getLength());

+	}

 

-    @Override

-    public void collect(Object input) throws HiveException {

-        try {

-            serializeResult(input);

-        } catch (IOException e) {

-            throw new HiveException(e);

-        } catch (SerDeException e) {

-            throw new HiveException(e);

-        }

-    }

+	@Override

+	public void collect(Object input) throws HiveException {

+		try {

+			serializeResult(input);

+		} catch (IOException e) {

+			throw new HiveException(e);

+		} catch (SerDeException e) {

+			throw new HiveException(e);

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index 2f16573..6c1ac72 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -43,6 +43,7 @@
 import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;

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

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

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

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

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

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

@@ -68,6 +69,7 @@
 import edu.uci.ics.hyracks.api.job.JobId;

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

 

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

 public class HyracksExecutionEngine implements IExecutionEngine {

 

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

@@ -191,7 +193,6 @@
 

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

                 translator.printOperators();

-                System.out.println("translate complete");

                 ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,

                         translator.getMetadataProvider());

 

@@ -223,7 +224,7 @@
     private void codeGen() throws AlgebricksException {

         // number of cpu cores in the cluster

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

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

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

         builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);

         builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);

         builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);

@@ -235,6 +236,7 @@
         builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);

         builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);

         builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);

+        builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);

 

         jobSpec = compiler.createJob(null);

 

@@ -244,9 +246,7 @@
             policyStr = "PIPELINING";

         Policy policyValue = Policy.valueOf(policyStr);

         jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));

-

-        // execute the job

-        System.out.println(jobSpec.toString());

+        jobSpec.setUseConnectorPolicyForScheduling(false);

     }

 

     @Override

@@ -341,7 +341,6 @@
                 // remove map-reduce branches in condition task

                 ConditionalTask condition = (ConditionalTask) task;

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

-                boolean existMR = false;

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

                     Task branch = branches.get(i);

                     if (branch instanceof MapRedTask) {

@@ -436,9 +435,7 @@
 

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

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

-            System.out.println("have child tasks!!");

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

-                System.out.println(child.getClass().getName());

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

                 if (childMapOps == null)

                     continue;

@@ -515,16 +512,16 @@
         String ipAddress = conf.get("hive.hyracks.host");

         int port = Integer.parseInt(conf.get("hive.hyracks.port"));

         String applicationName = conf.get("hive.hyracks.app");

-        System.out.println("connect to " + ipAddress + " " + port);

+        //System.out.println("connect to " + ipAddress + " " + port);

 

         IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);

 

-        System.out.println("get connected");

+        //System.out.println("get connected");

         long start = System.currentTimeMillis();

         JobId jobId = hcc.startJob(applicationName, job);

         hcc.waitForCompletion(jobId);

 

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

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

         // call all leave nodes to end

         for (Operator leaf : leaveOps) {

             jobClose(leaf);

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java
index c64a39b..8f4c471 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/exec/IExecutionEngine.java
@@ -7,19 +7,19 @@
 

 public interface IExecutionEngine {

 

-    /**

-     * compile the job

-     * 

-     * @param rootTasks

-     *            : Hive MapReduce plan

-     * @return 0 pass, 1 fail

-     */

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

+	/**

+	 * compile the job

+	 * 

+	 * @param rootTasks

+	 *            : Hive MapReduce plan

+	 * @return 0 pass, 1 fail

+	 */

+	public int compileJob(List<Task<? extends Serializable>> rootTasks);

 

-    /**

-     * execute the job with latest compiled plan

-     * 

-     * @return

-     */

-    public int executeJob();

+	/**

+	 * execute the job with latest compiled plan

+	 * 

+	 * @return

+	 */

+	public int executeJob();

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java
index f3b76e4..9c2d463 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryAscComparatorFactory.java
@@ -3,32 +3,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveByteBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveByteBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveByteBinaryAscComparatorFactory INSTANCE = new HiveByteBinaryAscComparatorFactory();

+	public static HiveByteBinaryAscComparatorFactory INSTANCE = new HiveByteBinaryAscComparatorFactory();

 

-    private HiveByteBinaryAscComparatorFactory() {

-    }

+	private HiveByteBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private byte left;

-            private byte right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private byte left;

+			private byte right;

 

-            @Override

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

-                left = b1[s1];

-                right = b2[s2];

-                if (left > right)

-                    return 1;

-                else if (left == right)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = b1[s1];

+				right = b2[s2];

+				if (left > right)

+					return 1;

+				else if (left == right)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java
index 8d452dc..ee71655 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveByteBinaryDescComparatorFactory.java
@@ -3,31 +3,33 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveByteBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveByteBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveByteBinaryDescComparatorFactory INSTANCE = new HiveByteBinaryDescComparatorFactory();

+	public static HiveByteBinaryDescComparatorFactory INSTANCE = new HiveByteBinaryDescComparatorFactory();

 

-    private HiveByteBinaryDescComparatorFactory() {

-    }

+	private HiveByteBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private byte left;

-            private byte right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private byte left;

+			private byte right;

 

-            @Override

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

-                left = b1[s1];

-                right = b2[s2];

-                if (left > right)

-                    return -1;

-                else if (left == right)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = b1[s1];

+				right = b2[s2];

+				if (left > right)

+					return -1;

+				else if (left == right)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java
index 0b5350a..739e417 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryAscComparatorFactory.java
@@ -4,32 +4,36 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveDoubleBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveDoubleBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveDoubleBinaryAscComparatorFactory INSTANCE = new HiveDoubleBinaryAscComparatorFactory();

+	public static HiveDoubleBinaryAscComparatorFactory INSTANCE = new HiveDoubleBinaryAscComparatorFactory();

 

-    private HiveDoubleBinaryAscComparatorFactory() {

-    }

+	private HiveDoubleBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private double left;

-            private double right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private double left;

+			private double right;

 

-            @Override

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

-                left = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b1, s1));

-                right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2, s2));

-                if (left > right)

-                    return 1;

-                else if (left == right)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = Double.longBitsToDouble(LazyUtils

+						.byteArrayToLong(b1, s1));

+				right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2,

+						s2));

+				if (left > right)

+					return 1;

+				else if (left == right)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java
index 2405956..0424c9f 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveDoubleBinaryDescComparatorFactory.java
@@ -4,32 +4,36 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveDoubleBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveDoubleBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveDoubleBinaryDescComparatorFactory INSTANCE = new HiveDoubleBinaryDescComparatorFactory();

+	public static HiveDoubleBinaryDescComparatorFactory INSTANCE = new HiveDoubleBinaryDescComparatorFactory();

 

-    private HiveDoubleBinaryDescComparatorFactory() {

-    }

+	private HiveDoubleBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private double left;

-            private double right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private double left;

+			private double right;

 

-            @Override

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

-                left = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b1, s1));

-                right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2, s2));

-                if (left > right)

-                    return -1;

-                else if (left == right)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = Double.longBitsToDouble(LazyUtils

+						.byteArrayToLong(b1, s1));

+				right = Double.longBitsToDouble(LazyUtils.byteArrayToLong(b2,

+						s2));

+				if (left > right)

+					return -1;

+				else if (left == right)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java
index 05a43e6..08542a7 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryAscComparatorFactory.java
@@ -4,32 +4,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveFloatBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveFloatBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveFloatBinaryAscComparatorFactory INSTANCE = new HiveFloatBinaryAscComparatorFactory();

+	public static HiveFloatBinaryAscComparatorFactory INSTANCE = new HiveFloatBinaryAscComparatorFactory();

 

-    private HiveFloatBinaryAscComparatorFactory() {

-    }

+	private HiveFloatBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private float left;

-            private float right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private float left;

+			private float right;

 

-            @Override

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

-                left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));

-                right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));

-                if (left > right)

-                    return 1;

-                else if (left == right)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));

+				right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));

+				if (left > right)

+					return 1;

+				else if (left == right)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java
index 2c44f97..513512e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveFloatBinaryDescComparatorFactory.java
@@ -4,32 +4,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveFloatBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveFloatBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveFloatBinaryDescComparatorFactory INSTANCE = new HiveFloatBinaryDescComparatorFactory();

+	public static HiveFloatBinaryDescComparatorFactory INSTANCE = new HiveFloatBinaryDescComparatorFactory();

 

-    private HiveFloatBinaryDescComparatorFactory() {

-    }

+	private HiveFloatBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private float left;

-            private float right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private float left;

+			private float right;

 

-            @Override

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

-                left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));

-                right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));

-                if (left > right)

-                    return -1;

-                else if (left == right)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b1, s1));

+				right = Float.intBitsToFloat(LazyUtils.byteArrayToInt(b2, s2));

+				if (left > right)

+					return -1;

+				else if (left == right)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java
index 0127791..947f30f 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryAscComparatorFactory.java
@@ -5,36 +5,40 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveIntegerBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveIntegerBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static final HiveIntegerBinaryAscComparatorFactory INSTANCE = new HiveIntegerBinaryAscComparatorFactory();

+	public static final HiveIntegerBinaryAscComparatorFactory INSTANCE = new HiveIntegerBinaryAscComparatorFactory();

 

-    private HiveIntegerBinaryAscComparatorFactory() {

-    }

+	private HiveIntegerBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VInt left = new VInt();

-            private VInt right = new VInt();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VInt left = new VInt();

+			private VInt right = new VInt();

 

-            @Override

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

-                LazyUtils.readVInt(b1, s1, left);

-                LazyUtils.readVInt(b2, s2, right);

+			@Override

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

+					int l2) {

+				LazyUtils.readVInt(b1, s1, left);

+				LazyUtils.readVInt(b2, s2, right);

 

-                if (left.length != l1 || right.length != l2)

-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "

-                            + left.length + "," + right.length + " expected " + l1 + "," + l2);

+				if (left.length != l1 || right.length != l2)

+					throw new IllegalArgumentException(

+							"length mismatch in int comparator function actual: "

+									+ left.length + "," + right.length

+									+ " expected " + l1 + "," + l2);

 

-                if (left.value > right.value)

-                    return 1;

-                else if (left.value == right.value)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+				if (left.value > right.value)

+					return 1;

+				else if (left.value == right.value)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java
index 5116337..7614aa1 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveIntegerBinaryDescComparatorFactory.java
@@ -5,34 +5,37 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveIntegerBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveIntegerBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static final HiveIntegerBinaryDescComparatorFactory INSTANCE = new HiveIntegerBinaryDescComparatorFactory();

+	public static final HiveIntegerBinaryDescComparatorFactory INSTANCE = new HiveIntegerBinaryDescComparatorFactory();

 

-    private HiveIntegerBinaryDescComparatorFactory() {

-    }

+	private HiveIntegerBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VInt left = new VInt();

-            private VInt right = new VInt();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VInt left = new VInt();

+			private VInt right = new VInt();

 

-            @Override

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

-                LazyUtils.readVInt(b1, s1, left);

-                LazyUtils.readVInt(b2, s2, right);

-                if (left.length != l1 || right.length != l2)

-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "

-                            + left.length + " expected " + l1);

-                if (left.value > right.value)

-                    return -1;

-                else if (left.value == right.value)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				LazyUtils.readVInt(b1, s1, left);

+				LazyUtils.readVInt(b2, s2, right);

+				if (left.length != l1 || right.length != l2)

+					throw new IllegalArgumentException(

+							"length mismatch in int comparator function actual: "

+									+ left.length + " expected " + l1);

+				if (left.value > right.value)

+					return -1;

+				else if (left.value == right.value)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java
index fa416a9..f5f3473 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryAscComparatorFactory.java
@@ -5,34 +5,37 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveLongBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveLongBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static final HiveLongBinaryAscComparatorFactory INSTANCE = new HiveLongBinaryAscComparatorFactory();

+	public static final HiveLongBinaryAscComparatorFactory INSTANCE = new HiveLongBinaryAscComparatorFactory();

 

-    private HiveLongBinaryAscComparatorFactory() {

-    }

+	private HiveLongBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VLong left = new VLong();

-            private VLong right = new VLong();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VLong left = new VLong();

+			private VLong right = new VLong();

 

-            @Override

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

-                LazyUtils.readVLong(b1, s1, left);

-                LazyUtils.readVLong(b2, s2, right);

-                if (left.length != l1 || right.length != l2)

-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "

-                            + left.length + " expected " + l1);

-                if (left.value > right.value)

-                    return 1;

-                else if (left.value == right.value)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				LazyUtils.readVLong(b1, s1, left);

+				LazyUtils.readVLong(b2, s2, right);

+				if (left.length != l1 || right.length != l2)

+					throw new IllegalArgumentException(

+							"length mismatch in int comparator function actual: "

+									+ left.length + " expected " + l1);

+				if (left.value > right.value)

+					return 1;

+				else if (left.value == right.value)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java
index e72dc62..b878b22 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveLongBinaryDescComparatorFactory.java
@@ -5,34 +5,37 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveLongBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveLongBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static final HiveLongBinaryDescComparatorFactory INSTANCE = new HiveLongBinaryDescComparatorFactory();

+	public static final HiveLongBinaryDescComparatorFactory INSTANCE = new HiveLongBinaryDescComparatorFactory();

 

-    private HiveLongBinaryDescComparatorFactory() {

-    }

+	private HiveLongBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VLong left = new VLong();

-            private VLong right = new VLong();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VLong left = new VLong();

+			private VLong right = new VLong();

 

-            @Override

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

-                LazyUtils.readVLong(b1, s1, left);

-                LazyUtils.readVLong(b2, s2, right);

-                if (left.length != l1 || right.length != l2)

-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "

-                            + left.length + " expected " + l1);

-                if (left.value > right.value)

-                    return -1;

-                else if (left.value == right.value)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				LazyUtils.readVLong(b1, s1, left);

+				LazyUtils.readVLong(b2, s2, right);

+				if (left.length != l1 || right.length != l2)

+					throw new IllegalArgumentException(

+							"length mismatch in int comparator function actual: "

+									+ left.length + " expected " + l1);

+				if (left.value > right.value)

+					return -1;

+				else if (left.value == right.value)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java
index a3745fa..8d55cdb 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryAscComparatorFactory.java
@@ -4,32 +4,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveShortBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveShortBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveShortBinaryAscComparatorFactory INSTANCE = new HiveShortBinaryAscComparatorFactory();

+	public static HiveShortBinaryAscComparatorFactory INSTANCE = new HiveShortBinaryAscComparatorFactory();

 

-    private HiveShortBinaryAscComparatorFactory() {

-    }

+	private HiveShortBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private short left;

-            private short right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private short left;

+			private short right;

 

-            @Override

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

-                left = LazyUtils.byteArrayToShort(b1, s1);

-                right = LazyUtils.byteArrayToShort(b2, s2);

-                if (left > right)

-                    return 1;

-                else if (left == right)

-                    return 0;

-                else

-                    return -1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = LazyUtils.byteArrayToShort(b1, s1);

+				right = LazyUtils.byteArrayToShort(b2, s2);

+				if (left > right)

+					return 1;

+				else if (left == right)

+					return 0;

+				else

+					return -1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java
index 44d3f43..4e8dde6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveShortBinaryDescComparatorFactory.java
@@ -4,32 +4,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveShortBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveShortBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveShortBinaryDescComparatorFactory INSTANCE = new HiveShortBinaryDescComparatorFactory();

+	public static HiveShortBinaryDescComparatorFactory INSTANCE = new HiveShortBinaryDescComparatorFactory();

 

-    private HiveShortBinaryDescComparatorFactory() {

-    }

+	private HiveShortBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private short left;

-            private short right;

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private short left;

+			private short right;

 

-            @Override

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

-                left = LazyUtils.byteArrayToShort(b1, s1);

-                right = LazyUtils.byteArrayToShort(b2, s2);

-                if (left > right)

-                    return -1;

-                else if (left == right)

-                    return 0;

-                else

-                    return 1;

-            }

-        };

-    }

+			@Override

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

+					int l2) {

+				left = LazyUtils.byteArrayToShort(b1, s1);

+				right = LazyUtils.byteArrayToShort(b2, s2);

+				if (left > right)

+					return -1;

+				else if (left == right)

+					return 0;

+				else

+					return 1;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java
index 6da9716..a334ecf 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryAscComparatorFactory.java
@@ -7,34 +7,40 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveStringBinaryAscComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveStringBinaryAscComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveStringBinaryAscComparatorFactory INSTANCE = new HiveStringBinaryAscComparatorFactory();

+	public static HiveStringBinaryAscComparatorFactory INSTANCE = new HiveStringBinaryAscComparatorFactory();

 

-    private HiveStringBinaryAscComparatorFactory() {

-    }

+	private HiveStringBinaryAscComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VInt leftLen = new VInt();

-            private VInt rightLen = new VInt();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VInt leftLen = new VInt();

+			private VInt rightLen = new VInt();

 

-            @Override

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

-                LazyUtils.readVInt(b1, s1, leftLen);

-                LazyUtils.readVInt(b2, s2, rightLen);

+			@Override

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

+					int l2) {

+				LazyUtils.readVInt(b1, s1, leftLen);

+				LazyUtils.readVInt(b2, s2, rightLen);

 

-                if (leftLen.value + leftLen.length != l1 || rightLen.value + rightLen.length != l2)

-                    throw new IllegalStateException("parse string: length mismatch, expected "

-                            + (leftLen.value + leftLen.length) + ", " + (rightLen.value + rightLen.length)

-                            + " but get " + l1 + ", " + l2);

+				if (leftLen.value + leftLen.length != l1

+						|| rightLen.value + rightLen.length != l2)

+					throw new IllegalStateException(

+							"parse string: length mismatch, expected "

+									+ (leftLen.value + leftLen.length) + ", "

+									+ (rightLen.value + rightLen.length)

+									+ " but get " + l1 + ", " + l2);

 

-                return Text.Comparator.compareBytes(b1, s1 + leftLen.length, l1 - leftLen.length, b2, s2

-                        + rightLen.length, l2 - rightLen.length);

-            }

-        };

-    }

+				return Text.Comparator.compareBytes(b1, s1 + leftLen.length, l1

+						- leftLen.length, b2, s2 + rightLen.length, l2

+						- rightLen.length);

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java
index c579711..e00b58e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/comparator/HiveStringBinaryDescComparatorFactory.java
@@ -7,33 +7,39 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;

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

 

-public class HiveStringBinaryDescComparatorFactory implements IBinaryComparatorFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveStringBinaryDescComparatorFactory implements

+		IBinaryComparatorFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveStringBinaryDescComparatorFactory INSTANCE = new HiveStringBinaryDescComparatorFactory();

+	public static HiveStringBinaryDescComparatorFactory INSTANCE = new HiveStringBinaryDescComparatorFactory();

 

-    private HiveStringBinaryDescComparatorFactory() {

-    }

+	private HiveStringBinaryDescComparatorFactory() {

+	}

 

-    @Override

-    public IBinaryComparator createBinaryComparator() {

-        return new IBinaryComparator() {

-            private VInt leftLen = new VInt();

-            private VInt rightLen = new VInt();

+	@Override

+	public IBinaryComparator createBinaryComparator() {

+		return new IBinaryComparator() {

+			private VInt leftLen = new VInt();

+			private VInt rightLen = new VInt();

 

-            @Override

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

-                LazyUtils.readVInt(b1, s1, leftLen);

-                LazyUtils.readVInt(b2, s2, rightLen);

+			@Override

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

+					int l2) {

+				LazyUtils.readVInt(b1, s1, leftLen);

+				LazyUtils.readVInt(b2, s2, rightLen);

 

-                if (leftLen.value + leftLen.length != l1 || rightLen.value + rightLen.length != l2)

-                    throw new IllegalStateException("parse string: length mismatch, expected "

-                            + (leftLen.value + leftLen.length) + ", " + (rightLen.value + rightLen.length)

-                            + " but get " + l1 + ", " + l2);

+				if (leftLen.value + leftLen.length != l1

+						|| rightLen.value + rightLen.length != l2)

+					throw new IllegalStateException(

+							"parse string: length mismatch, expected "

+									+ (leftLen.value + leftLen.length) + ", "

+									+ (rightLen.value + rightLen.length)

+									+ " but get " + l1 + ", " + l2);

 

-                return -WritableComparator.compareBytes(b1, s1 + leftLen.length, l1 - leftLen.length, b2, s2

-                        + rightLen.length, l2 - rightLen.length);

-            }

-        };

-    }

+				return -WritableComparator.compareBytes(b1,

+						s1 + leftLen.length, l1 - leftLen.length, b2, s2

+								+ rightLen.length, l2 - rightLen.length);

+			}

+		};

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
index 365379a..c6078ca 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionFactory.java
@@ -32,337 +32,350 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyAggregateFunctionFactory;

 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;

 

-public class AggregationFunctionFactory implements ICopyAggregateFunctionFactory {

+public class AggregationFunctionFactory implements

+		ICopyAggregateFunctionFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    /**

-     * list of parameters' serialization

-     */

-    private List<String> parametersSerialization = new ArrayList<String>();

+	/**

+	 * list of parameters' serialization

+	 */

+	private List<String> parametersSerialization = new ArrayList<String>();

 

-    /**

-     * the name of the udf

-     */

-    private String genericUDAFName;

+	/**

+	 * the name of the udf

+	 */

+	private String genericUDAFName;

 

-    /**

-     * aggregation mode

-     */

-    private GenericUDAFEvaluator.Mode mode;

+	/**

+	 * aggregation mode

+	 */

+	private GenericUDAFEvaluator.Mode mode;

 

-    /**

-     * list of type info

-     */

-    private List<TypeInfo> types = new ArrayList<TypeInfo>();

+	/**

+	 * list of type info

+	 */

+	private List<TypeInfo> types = new ArrayList<TypeInfo>();

 

-    /**

-     * distinct or not

-     */

-    private boolean distinct;

+	/**

+	 * distinct or not

+	 */

+	private boolean distinct;

 

-    /**

-     * the schema of incoming rows

-     */

-    private Schema rowSchema;

+	/**

+	 * the schema of incoming rows

+	 */

+	private Schema rowSchema;

 

-    /**

-     * list of parameters

-     */

-    private transient List<ExprNodeDesc> parametersOrigin;

+	/**

+	 * list of parameters

+	 */

+	private transient List<ExprNodeDesc> parametersOrigin;

 

-    /**

-     * row inspector

-     */

-    private transient ObjectInspector rowInspector = null;

+	/**

+	 * row inspector

+	 */

+	private transient ObjectInspector rowInspector = null;

 

-    /**

-     * output object inspector

-     */

-    private transient ObjectInspector outputInspector = null;

+	/**

+	 * output object inspector

+	 */

+	private transient ObjectInspector outputInspector = null;

 

-    /**

-     * output object inspector

-     */

-    private transient ObjectInspector outputInspectorPartial = null;

+	/**

+	 * output object inspector

+	 */

+	private transient ObjectInspector outputInspectorPartial = null;

 

-    /**

-     * parameter inspectors

-     */

-    private transient ObjectInspector[] parameterInspectors = null;

+	/**

+	 * parameter inspectors

+	 */

+	private transient ObjectInspector[] parameterInspectors = null;

 

-    /**

-     * expression desc

-     */

-    private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();

+	/**

+	 * expression desc

+	 */

+	private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();

 

-    /**

-     * evaluators

-     */

-    private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();

+	/**

+	 * evaluators

+	 */

+	private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();

 

-    /**

-     * cached parameter objects

-     */

-    private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();

+	/**

+	 * cached parameter objects

+	 */

+	private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();

 

-    /**

-     * cached row object: one per thread

-     */

-    private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();

+	/**

+	 * cached row object: one per thread

+	 */

+	private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();

 

-    /**

-     * we only use lazy serde to do serialization

-     */

-    private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();

+	/**

+	 * we only use lazy serde to do serialization

+	 */

+	private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();

 

-    /**

-     * udaf evaluators

-     */

-    private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();

+	/**

+	 * udaf evaluators

+	 */

+	private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();

 

-    /**

-     * udaf evaluators

-     */

-    private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();

+	/**

+	 * udaf evaluators

+	 */

+	private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();

 

-    /**

-     * aggregation function desc

-     */

-    private transient AggregationDesc aggregator;

+	/**

+	 * aggregation function desc

+	 */

+	private transient AggregationDesc aggregator;

 

-    /**

-     * 

-     * @param aggregator

-     *            Algebricks function call expression

-     * @param oi

-     *            schema

-     */

-    public AggregationFunctionFactory(AggregateFunctionCallExpression expression, Schema oi,

-            IVariableTypeEnvironment env) throws AlgebricksException {

+	/**

+	 * 

+	 * @param aggregator

+	 *            Algebricks function call expression

+	 * @param oi

+	 *            schema

+	 */

+	public AggregationFunctionFactory(

+			AggregateFunctionCallExpression expression, Schema oi,

+			IVariableTypeEnvironment env) throws AlgebricksException {

 

-        try {

-            aggregator = (AggregationDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e.getMessage());

-        }

-        init(aggregator.getParameters(), aggregator.getGenericUDAFName(), aggregator.getMode(),

-                aggregator.getDistinct(), oi);

-    }

+		try {

+			aggregator = (AggregationDesc) ExpressionTranslator

+					.getHiveExpression(expression, env);

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e.getMessage());

+		}

+		init(aggregator.getParameters(), aggregator.getGenericUDAFName(),

+				aggregator.getMode(), aggregator.getDistinct(), oi);

+	}

 

-    /**

-     * constructor of aggregation function factory

-     * 

-     * @param inputs

-     * @param name

-     * @param udafMode

-     * @param distinct

-     * @param oi

-     */

-    private void init(List<ExprNodeDesc> inputs, String name, GenericUDAFEvaluator.Mode udafMode, boolean distinct,

-            Schema oi) {

-        parametersOrigin = inputs;

-        genericUDAFName = name;

-        mode = udafMode;

-        this.distinct = distinct;

-        rowSchema = oi;

+	/**

+	 * constructor of aggregation function factory

+	 * 

+	 * @param inputs

+	 * @param name

+	 * @param udafMode

+	 * @param distinct

+	 * @param oi

+	 */

+	private void init(List<ExprNodeDesc> inputs, String name,

+			GenericUDAFEvaluator.Mode udafMode, boolean distinct, Schema oi) {

+		parametersOrigin = inputs;

+		genericUDAFName = name;

+		mode = udafMode;

+		this.distinct = distinct;

+		rowSchema = oi;

 

-        for (ExprNodeDesc input : inputs) {

-            TypeInfo type = input.getTypeInfo();

-            if (type instanceof StructTypeInfo) {

-                types.add(TypeInfoFactory.doubleTypeInfo);

-            } else

-                types.add(type);

+		for (ExprNodeDesc input : inputs) {

+			TypeInfo type = input.getTypeInfo();

+			if (type instanceof StructTypeInfo) {

+				types.add(TypeInfoFactory.doubleTypeInfo);

+			} else

+				types.add(type);

 

-            String s = Utilities.serializeExpression(input);

-            parametersSerialization.add(s);

-        }

-    }

+			String s = Utilities.serializeExpression(input);

+			parametersSerialization.add(s);

+		}

+	}

 

-    @Override

-    public synchronized ICopyAggregateFunction createAggregateFunction(IDataOutputProvider provider)

-            throws AlgebricksException {

-        if (parametersOrigin == null) {

-            Configuration config = new Configuration();

-            config.setClassLoader(this.getClass().getClassLoader());

-            /**

-             * in case of class.forname(...) call in hive code

-             */

-            Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());

+	@Override

+	public synchronized ICopyAggregateFunction createAggregateFunction(

+			IDataOutputProvider provider) throws AlgebricksException {

+		if (parametersOrigin == null) {

+			Configuration config = new Configuration();

+			config.setClassLoader(this.getClass().getClassLoader());

+			/**

+			 * in case of class.forname(...) call in hive code

+			 */

+			Thread.currentThread().setContextClassLoader(

+					this.getClass().getClassLoader());

 

-            parametersOrigin = new ArrayList<ExprNodeDesc>();

-            for (String serialization : parametersSerialization) {

-                parametersOrigin.add(Utilities.deserializeExpression(serialization, config));

-            }

-        }

+			parametersOrigin = new ArrayList<ExprNodeDesc>();

+			for (String serialization : parametersSerialization) {

+				parametersOrigin.add(Utilities.deserializeExpression(

+						serialization, config));

+			}

+		}

 

-        /**

-         * exprs

-         */

-        if (parameterExprs == null)

-            parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();

+		/**

+		 * exprs

+		 */

+		if (parameterExprs == null)

+			parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();

 

-        /**

-         * evaluators

-         */

-        if (evaluators == null)

-            evaluators = new HashMap<Long, ExprNodeEvaluator[]>();

+		/**

+		 * evaluators

+		 */

+		if (evaluators == null)

+			evaluators = new HashMap<Long, ExprNodeEvaluator[]>();

 

-        /**

-         * cached parameter objects

-         */

-        if (cachedParameters == null)

-            cachedParameters = new HashMap<Long, Object[]>();

+		/**

+		 * cached parameter objects

+		 */

+		if (cachedParameters == null)

+			cachedParameters = new HashMap<Long, Object[]>();

 

-        /**

-         * cached row object: one per thread

-         */

-        if (cachedRowObjects == null)

-            cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();

+		/**

+		 * cached row object: one per thread

+		 */

+		if (cachedRowObjects == null)

+			cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();

 

-        /**

-         * we only use lazy serde to do serialization

-         */

-        if (serDe == null)

-            serDe = new HashMap<Long, SerDe>();

+		/**

+		 * we only use lazy serde to do serialization

+		 */

+		if (serDe == null)

+			serDe = new HashMap<Long, SerDe>();

 

-        /**

-         * UDAF functions

-         */

-        if (udafsComplete == null)

-            udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();

+		/**

+		 * UDAF functions

+		 */

+		if (udafsComplete == null)

+			udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();

 

-        /**

-         * UDAF functions

-         */

-        if (udafsPartial == null)

-            udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();

+		/**

+		 * UDAF functions

+		 */

+		if (udafsPartial == null)

+			udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();

 

-        if (parameterInspectors == null)

-            parameterInspectors = new ObjectInspector[parametersOrigin.size()];

+		if (parameterInspectors == null)

+			parameterInspectors = new ObjectInspector[parametersOrigin.size()];

 

-        if (rowInspector == null)

-            rowInspector = rowSchema.toObjectInspector();

+		if (rowInspector == null)

+			rowInspector = rowSchema.toObjectInspector();

 

-        // get current thread id

-        long threadId = Thread.currentThread().getId();

+		// get current thread id

+		long threadId = Thread.currentThread().getId();

 

-        /**

-         * expressions, expressions are thread local

-         */

-        List<ExprNodeDesc> parameters = parameterExprs.get(threadId);

-        if (parameters == null) {

-            parameters = new ArrayList<ExprNodeDesc>();

-            for (ExprNodeDesc parameter : parametersOrigin)

-                parameters.add(parameter.clone());

-            parameterExprs.put(threadId, parameters);

-        }

+		/**

+		 * expressions, expressions are thread local

+		 */

+		List<ExprNodeDesc> parameters = parameterExprs.get(threadId);

+		if (parameters == null) {

+			parameters = new ArrayList<ExprNodeDesc>();

+			for (ExprNodeDesc parameter : parametersOrigin)

+				parameters.add(parameter.clone());

+			parameterExprs.put(threadId, parameters);

+		}

 

-        /**

-         * cached parameter objects

-         */

-        Object[] cachedParas = cachedParameters.get(threadId);

-        if (cachedParas == null) {

-            cachedParas = new Object[parameters.size()];

-            cachedParameters.put(threadId, cachedParas);

-        }

+		/**

+		 * cached parameter objects

+		 */

+		Object[] cachedParas = cachedParameters.get(threadId);

+		if (cachedParas == null) {

+			cachedParas = new Object[parameters.size()];

+			cachedParameters.put(threadId, cachedParas);

+		}

 

-        /**

-         * cached row object: one per thread

-         */

-        LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects.get(threadId);

-        if (cachedRowObject == null) {

-            cachedRowObject = LazyFactory.createLazyObject(rowInspector);

-            cachedRowObjects.put(threadId, cachedRowObject);

-        }

+		/**

+		 * cached row object: one per thread

+		 */

+		LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects

+				.get(threadId);

+		if (cachedRowObject == null) {

+			cachedRowObject = LazyFactory.createLazyObject(rowInspector);

+			cachedRowObjects.put(threadId, cachedRowObject);

+		}

 

-        /**

-         * we only use lazy serde to do serialization

-         */

-        SerDe lazySer = serDe.get(threadId);

-        if (lazySer == null) {

-            lazySer = new LazySerDe();

-            serDe.put(threadId, lazySer);

-        }

+		/**

+		 * we only use lazy serde to do serialization

+		 */

+		SerDe lazySer = serDe.get(threadId);

+		if (lazySer == null) {

+			lazySer = new LazySerDe();

+			serDe.put(threadId, lazySer);

+		}

 

-        /**

-         * evaluators

-         */

-        ExprNodeEvaluator[] evals = evaluators.get(threadId);

-        if (evals == null) {

-            evals = new ExprNodeEvaluator[parameters.size()];

-            evaluators.put(threadId, evals);

-        }

+		/**

+		 * evaluators

+		 */

+		ExprNodeEvaluator[] evals = evaluators.get(threadId);

+		if (evals == null) {

+			evals = new ExprNodeEvaluator[parameters.size()];

+			evaluators.put(threadId, evals);

+		}

 

-        GenericUDAFEvaluator udafPartial;

-        GenericUDAFEvaluator udafComplete;

+		GenericUDAFEvaluator udafPartial;

+		GenericUDAFEvaluator udafComplete;

 

-        // initialize object inspectors

-        try {

-            /**

-             * evaluators, udf, object inpsectors are shared in one thread

-             */

-            for (int i = 0; i < evals.length; i++) {

-                if (evals[i] == null) {

-                    evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));

-                    if (parameterInspectors[i] == null) {

-                        parameterInspectors[i] = evals[i].initialize(rowInspector);

-                    } else {

-                        evals[i].initialize(rowInspector);

-                    }

-                }

-            }

+		// initialize object inspectors

+		try {

+			/**

+			 * evaluators, udf, object inpsectors are shared in one thread

+			 */

+			for (int i = 0; i < evals.length; i++) {

+				if (evals[i] == null) {

+					evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));

+					if (parameterInspectors[i] == null) {

+						parameterInspectors[i] = evals[i]

+								.initialize(rowInspector);

+					} else {

+						evals[i].initialize(rowInspector);

+					}

+				}

+			}

 

-            udafComplete = udafsComplete.get(threadId);

-            if (udafComplete == null) {

-                try {

-                    udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);

-                } catch (HiveException e) {

-                    throw new AlgebricksException(e);

-                }

-                udafsComplete.put(threadId, udafComplete);

-                udafComplete.init(mode, parameterInspectors);

-            }

+			udafComplete = udafsComplete.get(threadId);

+			if (udafComplete == null) {

+				try {

+					udafComplete = FunctionRegistry.getGenericUDAFEvaluator(

+							genericUDAFName, types, distinct, false);

+				} catch (HiveException e) {

+					throw new AlgebricksException(e);

+				}

+				udafsComplete.put(threadId, udafComplete);

+				udafComplete.init(mode, parameterInspectors);

+			}

 

-            // multiple stage group by, determined by the mode parameter

-            if (outputInspector == null)

-                outputInspector = udafComplete.init(mode, parameterInspectors);

+			// multiple stage group by, determined by the mode parameter

+			if (outputInspector == null)

+				outputInspector = udafComplete.init(mode, parameterInspectors);

 

-            // initial partial gby udaf

-            GenericUDAFEvaluator.Mode partialMode;

-            // adjust mode for external groupby

-            if (mode == GenericUDAFEvaluator.Mode.COMPLETE)

-                partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;

-            else if (mode == GenericUDAFEvaluator.Mode.FINAL)

-                partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;

-            else

-                partialMode = mode;

-            udafPartial = udafsPartial.get(threadId);

-            if (udafPartial == null) {

-                try {

-                    udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);

-                } catch (HiveException e) {

-                    throw new AlgebricksException(e);

-                }

-                udafPartial.init(partialMode, parameterInspectors);

-                udafsPartial.put(threadId, udafPartial);

-            }

+			// initial partial gby udaf

+			GenericUDAFEvaluator.Mode partialMode;

+			// adjust mode for external groupby

+			if (mode == GenericUDAFEvaluator.Mode.COMPLETE)

+				partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;

+			else if (mode == GenericUDAFEvaluator.Mode.FINAL)

+				partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;

+			else

+				partialMode = mode;

+			udafPartial = udafsPartial.get(threadId);

+			if (udafPartial == null) {

+				try {

+					udafPartial = FunctionRegistry.getGenericUDAFEvaluator(

+							genericUDAFName, types, distinct, false);

+				} catch (HiveException e) {

+					throw new AlgebricksException(e);

+				}

+				udafPartial.init(partialMode, parameterInspectors);

+				udafsPartial.put(threadId, udafPartial);

+			}

 

-            // multiple stage group by, determined by the mode parameter

-            if (outputInspectorPartial == null)

-                outputInspectorPartial = udafPartial.init(partialMode, parameterInspectors);

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e);

-        }

+			// multiple stage group by, determined by the mode parameter

+			if (outputInspectorPartial == null)

+				outputInspectorPartial = udafPartial.init(partialMode,

+						parameterInspectors);

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e);

+		}

 

-        return new AggregationFunctionEvaluator(parameters, types, genericUDAFName, mode, distinct, rowInspector,

-                provider.getDataOutput(), evals, parameterInspectors, cachedParas, lazySer, cachedRowObject,

-                udafPartial, udafComplete, outputInspector, outputInspectorPartial);

-    }

+		return new AggregationFunctionEvaluator(parameters, types,

+				genericUDAFName, mode, distinct, rowInspector,

+				provider.getDataOutput(), evals, parameterInspectors,

+				cachedParas, lazySer, cachedRowObject, udafPartial,

+				udafComplete, outputInspector, outputInspectorPartial);

+	}

 

-    public String toString() {

-        return "aggregation function expression evaluator factory: " + this.genericUDAFName;

-    }

+	public String toString() {

+		return "aggregation function expression evaluator factory: "

+				+ this.genericUDAFName;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
index 7a48319..73717a3 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/AggregationFunctionSerializableFactory.java
@@ -31,337 +31,351 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunctionFactory;
 
-public class AggregationFunctionSerializableFactory implements ICopySerializableAggregateFunctionFactory {
+public class AggregationFunctionSerializableFactory implements
+		ICopySerializableAggregateFunctionFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    /**
-     * list of parameters' serialization
-     */
-    private List<String> parametersSerialization = new ArrayList<String>();
+	/**
+	 * list of parameters' serialization
+	 */
+	private List<String> parametersSerialization = new ArrayList<String>();
 
-    /**
-     * the name of the udf
-     */
-    private String genericUDAFName;
+	/**
+	 * the name of the udf
+	 */
+	private String genericUDAFName;
 
-    /**
-     * aggregation mode
-     */
-    private GenericUDAFEvaluator.Mode mode;
+	/**
+	 * aggregation mode
+	 */
+	private GenericUDAFEvaluator.Mode mode;
 
-    /**
-     * list of type info
-     */
-    private List<TypeInfo> types = new ArrayList<TypeInfo>();
+	/**
+	 * list of type info
+	 */
+	private List<TypeInfo> types = new ArrayList<TypeInfo>();
 
-    /**
-     * distinct or not
-     */
-    private boolean distinct;
+	/**
+	 * distinct or not
+	 */
+	private boolean distinct;
 
-    /**
-     * the schema of incoming rows
-     */
-    private Schema rowSchema;
+	/**
+	 * the schema of incoming rows
+	 */
+	private Schema rowSchema;
 
-    /**
-     * list of parameters
-     */
-    private transient List<ExprNodeDesc> parametersOrigin;
+	/**
+	 * list of parameters
+	 */
+	private transient List<ExprNodeDesc> parametersOrigin;
 
-    /**
-     * row inspector
-     */
-    private transient ObjectInspector rowInspector = null;
+	/**
+	 * row inspector
+	 */
+	private transient ObjectInspector rowInspector = null;
 
-    /**
-     * output object inspector
-     */
-    private transient ObjectInspector outputInspector = null;
+	/**
+	 * output object inspector
+	 */
+	private transient ObjectInspector outputInspector = null;
 
-    /**
-     * output object inspector
-     */
-    private transient ObjectInspector outputInspectorPartial = null;
+	/**
+	 * output object inspector
+	 */
+	private transient ObjectInspector outputInspectorPartial = null;
 
-    /**
-     * parameter inspectors
-     */
-    private transient ObjectInspector[] parameterInspectors = null;
+	/**
+	 * parameter inspectors
+	 */
+	private transient ObjectInspector[] parameterInspectors = null;
 
-    /**
-     * expression desc
-     */
-    private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+	/**
+	 * expression desc
+	 */
+	private transient HashMap<Long, List<ExprNodeDesc>> parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
 
-    /**
-     * evaluators
-     */
-    private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+	/**
+	 * evaluators
+	 */
+	private transient HashMap<Long, ExprNodeEvaluator[]> evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
 
-    /**
-     * cached parameter objects
-     */
-    private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();
+	/**
+	 * cached parameter objects
+	 */
+	private transient HashMap<Long, Object[]> cachedParameters = new HashMap<Long, Object[]>();
 
-    /**
-     * cached row object: one per thread
-     */
-    private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+	/**
+	 * cached row object: one per thread
+	 */
+	private transient HashMap<Long, LazyObject<? extends ObjectInspector>> cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
 
-    /**
-     * we only use lazy serde to do serialization
-     */
-    private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();
+	/**
+	 * we only use lazy serde to do serialization
+	 */
+	private transient HashMap<Long, SerDe> serDe = new HashMap<Long, SerDe>();
 
-    /**
-     * udaf evaluators
-     */
-    private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+	/**
+	 * udaf evaluators
+	 */
+	private transient HashMap<Long, GenericUDAFEvaluator> udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
 
-    /**
-     * udaf evaluators
-     */
-    private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+	/**
+	 * udaf evaluators
+	 */
+	private transient HashMap<Long, GenericUDAFEvaluator> udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
 
-    /**
-     * aggregation function desc
-     */
-    private transient AggregationDesc aggregator;
+	/**
+	 * aggregation function desc
+	 */
+	private transient AggregationDesc aggregator;
 
-    /**
-     * 
-     * @param aggregator
-     *            Algebricks function call expression
-     * @param oi
-     *            schema
-     */
-    public AggregationFunctionSerializableFactory(AggregateFunctionCallExpression expression, Schema oi,
-            IVariableTypeEnvironment env) throws AlgebricksException {
+	/**
+	 * 
+	 * @param aggregator
+	 *            Algebricks function call expression
+	 * @param oi
+	 *            schema
+	 */
+	public AggregationFunctionSerializableFactory(
+			AggregateFunctionCallExpression expression, Schema oi,
+			IVariableTypeEnvironment env) throws AlgebricksException {
 
-        try {
-            aggregator = (AggregationDesc) ExpressionTranslator.getHiveExpression(expression, env);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new AlgebricksException(e.getMessage());
-        }
-        init(aggregator.getParameters(), aggregator.getGenericUDAFName(), aggregator.getMode(),
-                aggregator.getDistinct(), oi);
-    }
+		try {
+			aggregator = (AggregationDesc) ExpressionTranslator
+					.getHiveExpression(expression, env);
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new AlgebricksException(e.getMessage());
+		}
+		init(aggregator.getParameters(), aggregator.getGenericUDAFName(),
+				aggregator.getMode(), aggregator.getDistinct(), oi);
+	}
 
-    /**
-     * constructor of aggregation function factory
-     * 
-     * @param inputs
-     * @param name
-     * @param udafMode
-     * @param distinct
-     * @param oi
-     */
-    private void init(List<ExprNodeDesc> inputs, String name, GenericUDAFEvaluator.Mode udafMode, boolean distinct,
-            Schema oi) {
-        parametersOrigin = inputs;
-        genericUDAFName = name;
-        mode = udafMode;
-        this.distinct = distinct;
-        rowSchema = oi;
+	/**
+	 * constructor of aggregation function factory
+	 * 
+	 * @param inputs
+	 * @param name
+	 * @param udafMode
+	 * @param distinct
+	 * @param oi
+	 */
+	private void init(List<ExprNodeDesc> inputs, String name,
+			GenericUDAFEvaluator.Mode udafMode, boolean distinct, Schema oi) {
+		parametersOrigin = inputs;
+		genericUDAFName = name;
+		mode = udafMode;
+		this.distinct = distinct;
+		rowSchema = oi;
 
-        for (ExprNodeDesc input : inputs) {
-            TypeInfo type = input.getTypeInfo();
-            if (type instanceof StructTypeInfo) {
-                types.add(TypeInfoFactory.doubleTypeInfo);
-            } else
-                types.add(type);
+		for (ExprNodeDesc input : inputs) {
+			TypeInfo type = input.getTypeInfo();
+			if (type instanceof StructTypeInfo) {
+				types.add(TypeInfoFactory.doubleTypeInfo);
+			} else
+				types.add(type);
 
-            String s = Utilities.serializeExpression(input);
-            parametersSerialization.add(s);
-        }
-    }
+			String s = Utilities.serializeExpression(input);
+			parametersSerialization.add(s);
+		}
+	}
 
-    @Override
-    public synchronized ICopySerializableAggregateFunction createAggregateFunction() throws AlgebricksException {
-        if (parametersOrigin == null) {
-            Configuration config = new Configuration();
-            config.setClassLoader(this.getClass().getClassLoader());
-            /**
-             * in case of class.forname(...) call in hive code
-             */
-            Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+	@Override
+	public synchronized ICopySerializableAggregateFunction createAggregateFunction()
+			throws AlgebricksException {
+		if (parametersOrigin == null) {
+			Configuration config = new Configuration();
+			config.setClassLoader(this.getClass().getClassLoader());
+			/**
+			 * in case of class.forname(...) call in hive code
+			 */
+			Thread.currentThread().setContextClassLoader(
+					this.getClass().getClassLoader());
 
-            parametersOrigin = new ArrayList<ExprNodeDesc>();
-            for (String serialization : parametersSerialization) {
-                parametersOrigin.add(Utilities.deserializeExpression(serialization, config));
-            }
-        }
+			parametersOrigin = new ArrayList<ExprNodeDesc>();
+			for (String serialization : parametersSerialization) {
+				parametersOrigin.add(Utilities.deserializeExpression(
+						serialization, config));
+			}
+		}
 
-        /**
-         * exprs
-         */
-        if (parameterExprs == null)
-            parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
+		/**
+		 * exprs
+		 */
+		if (parameterExprs == null)
+			parameterExprs = new HashMap<Long, List<ExprNodeDesc>>();
 
-        /**
-         * evaluators
-         */
-        if (evaluators == null)
-            evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
+		/**
+		 * evaluators
+		 */
+		if (evaluators == null)
+			evaluators = new HashMap<Long, ExprNodeEvaluator[]>();
 
-        /**
-         * cached parameter objects
-         */
-        if (cachedParameters == null)
-            cachedParameters = new HashMap<Long, Object[]>();
+		/**
+		 * cached parameter objects
+		 */
+		if (cachedParameters == null)
+			cachedParameters = new HashMap<Long, Object[]>();
 
-        /**
-         * cached row object: one per thread
-         */
-        if (cachedRowObjects == null)
-            cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
+		/**
+		 * cached row object: one per thread
+		 */
+		if (cachedRowObjects == null)
+			cachedRowObjects = new HashMap<Long, LazyObject<? extends ObjectInspector>>();
 
-        /**
-         * we only use lazy serde to do serialization
-         */
-        if (serDe == null)
-            serDe = new HashMap<Long, SerDe>();
+		/**
+		 * we only use lazy serde to do serialization
+		 */
+		if (serDe == null)
+			serDe = new HashMap<Long, SerDe>();
 
-        /**
-         * UDAF functions
-         */
-        if (udafsComplete == null)
-            udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
+		/**
+		 * UDAF functions
+		 */
+		if (udafsComplete == null)
+			udafsComplete = new HashMap<Long, GenericUDAFEvaluator>();
 
-        /**
-         * UDAF functions
-         */
-        if (udafsPartial == null)
-            udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
+		/**
+		 * UDAF functions
+		 */
+		if (udafsPartial == null)
+			udafsPartial = new HashMap<Long, GenericUDAFEvaluator>();
 
-        if (parameterInspectors == null)
-            parameterInspectors = new ObjectInspector[parametersOrigin.size()];
+		if (parameterInspectors == null)
+			parameterInspectors = new ObjectInspector[parametersOrigin.size()];
 
-        if (rowInspector == null)
-            rowInspector = rowSchema.toObjectInspector();
+		if (rowInspector == null)
+			rowInspector = rowSchema.toObjectInspector();
 
-        // get current thread id
-        long threadId = Thread.currentThread().getId();
+		// get current thread id
+		long threadId = Thread.currentThread().getId();
 
-        /**
-         * expressions, expressions are thread local
-         */
-        List<ExprNodeDesc> parameters = parameterExprs.get(threadId);
-        if (parameters == null) {
-            parameters = new ArrayList<ExprNodeDesc>();
-            for (ExprNodeDesc parameter : parametersOrigin)
-                parameters.add(parameter.clone());
-            parameterExprs.put(threadId, parameters);
-        }
+		/**
+		 * expressions, expressions are thread local
+		 */
+		List<ExprNodeDesc> parameters = parameterExprs.get(threadId);
+		if (parameters == null) {
+			parameters = new ArrayList<ExprNodeDesc>();
+			for (ExprNodeDesc parameter : parametersOrigin)
+				parameters.add(parameter.clone());
+			parameterExprs.put(threadId, parameters);
+		}
 
-        /**
-         * cached parameter objects
-         */
-        Object[] cachedParas = cachedParameters.get(threadId);
-        if (cachedParas == null) {
-            cachedParas = new Object[parameters.size()];
-            cachedParameters.put(threadId, cachedParas);
-        }
+		/**
+		 * cached parameter objects
+		 */
+		Object[] cachedParas = cachedParameters.get(threadId);
+		if (cachedParas == null) {
+			cachedParas = new Object[parameters.size()];
+			cachedParameters.put(threadId, cachedParas);
+		}
 
-        /**
-         * cached row object: one per thread
-         */
-        LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects.get(threadId);
-        if (cachedRowObject == null) {
-            cachedRowObject = LazyFactory.createLazyObject(rowInspector);
-            cachedRowObjects.put(threadId, cachedRowObject);
-        }
+		/**
+		 * cached row object: one per thread
+		 */
+		LazyObject<? extends ObjectInspector> cachedRowObject = cachedRowObjects
+				.get(threadId);
+		if (cachedRowObject == null) {
+			cachedRowObject = LazyFactory.createLazyObject(rowInspector);
+			cachedRowObjects.put(threadId, cachedRowObject);
+		}
 
-        /**
-         * we only use lazy serde to do serialization
-         */
-        SerDe lazySer = serDe.get(threadId);
-        if (lazySer == null) {
-            lazySer = new LazySerDe();
-            serDe.put(threadId, lazySer);
-        }
+		/**
+		 * we only use lazy serde to do serialization
+		 */
+		SerDe lazySer = serDe.get(threadId);
+		if (lazySer == null) {
+			lazySer = new LazySerDe();
+			serDe.put(threadId, lazySer);
+		}
 
-        /**
-         * evaluators
-         */
-        ExprNodeEvaluator[] evals = evaluators.get(threadId);
-        if (evals == null) {
-            evals = new ExprNodeEvaluator[parameters.size()];
-            evaluators.put(threadId, evals);
-        }
+		/**
+		 * evaluators
+		 */
+		ExprNodeEvaluator[] evals = evaluators.get(threadId);
+		if (evals == null) {
+			evals = new ExprNodeEvaluator[parameters.size()];
+			evaluators.put(threadId, evals);
+		}
 
-        GenericUDAFEvaluator udafPartial;
-        GenericUDAFEvaluator udafComplete;
+		GenericUDAFEvaluator udafPartial;
+		GenericUDAFEvaluator udafComplete;
 
-        // initialize object inspectors
-        try {
-            /**
-             * evaluators, udf, object inpsectors are shared in one thread
-             */
-            for (int i = 0; i < evals.length; i++) {
-                if (evals[i] == null) {
-                    evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));
-                    if (parameterInspectors[i] == null) {
-                        parameterInspectors[i] = evals[i].initialize(rowInspector);
-                    } else {
-                        evals[i].initialize(rowInspector);
-                    }
-                }
-            }
+		// initialize object inspectors
+		try {
+			/**
+			 * evaluators, udf, object inpsectors are shared in one thread
+			 */
+			for (int i = 0; i < evals.length; i++) {
+				if (evals[i] == null) {
+					evals[i] = ExprNodeEvaluatorFactory.get(parameters.get(i));
+					if (parameterInspectors[i] == null) {
+						parameterInspectors[i] = evals[i]
+								.initialize(rowInspector);
+					} else {
+						evals[i].initialize(rowInspector);
+					}
+				}
+			}
 
-            udafComplete = udafsComplete.get(threadId);
-            if (udafComplete == null) {
-                try {
-                    udafComplete = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
-                } catch (HiveException e) {
-                    throw new AlgebricksException(e);
-                }
-                udafsComplete.put(threadId, udafComplete);
-                udafComplete.init(mode, parameterInspectors);
-            }
+			udafComplete = udafsComplete.get(threadId);
+			if (udafComplete == null) {
+				try {
+					udafComplete = FunctionRegistry.getGenericUDAFEvaluator(
+							genericUDAFName, types, distinct, false);
+				} catch (HiveException e) {
+					throw new AlgebricksException(e);
+				}
+				udafsComplete.put(threadId, udafComplete);
+				udafComplete.init(mode, parameterInspectors);
+			}
 
-            // multiple stage group by, determined by the mode parameter
-            if (outputInspector == null)
-                outputInspector = udafComplete.init(mode, parameterInspectors);
+			// multiple stage group by, determined by the mode parameter
+			if (outputInspector == null)
+				outputInspector = udafComplete.init(mode, parameterInspectors);
 
-            // initial partial gby udaf
-            GenericUDAFEvaluator.Mode partialMode;
-            // adjust mode for external groupby
-            if (mode == GenericUDAFEvaluator.Mode.COMPLETE)
-                partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;
-            else if (mode == GenericUDAFEvaluator.Mode.FINAL)
-                partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;
-            else
-                partialMode = mode;
-            udafPartial = udafsPartial.get(threadId);
-            if (udafPartial == null) {
-                try {
-                    udafPartial = FunctionRegistry.getGenericUDAFEvaluator(genericUDAFName, types, distinct, false);
-                } catch (HiveException e) {
-                    throw new AlgebricksException(e);
-                }
-                udafPartial.init(partialMode, parameterInspectors);
-                udafsPartial.put(threadId, udafPartial);
-            }
+			// initial partial gby udaf
+			GenericUDAFEvaluator.Mode partialMode;
+			// adjust mode for external groupby
+			if (mode == GenericUDAFEvaluator.Mode.COMPLETE)
+				partialMode = GenericUDAFEvaluator.Mode.PARTIAL1;
+			else if (mode == GenericUDAFEvaluator.Mode.FINAL)
+				partialMode = GenericUDAFEvaluator.Mode.PARTIAL2;
+			else
+				partialMode = mode;
+			udafPartial = udafsPartial.get(threadId);
+			if (udafPartial == null) {
+				try {
+					udafPartial = FunctionRegistry.getGenericUDAFEvaluator(
+							genericUDAFName, types, distinct, false);
+				} catch (HiveException e) {
+					throw new AlgebricksException(e);
+				}
+				udafPartial.init(partialMode, parameterInspectors);
+				udafsPartial.put(threadId, udafPartial);
+			}
 
-            // multiple stage group by, determined by the mode parameter
-            if (outputInspectorPartial == null)
-                outputInspectorPartial = udafPartial.init(partialMode, parameterInspectors);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new AlgebricksException(e);
-        }
+			// multiple stage group by, determined by the mode parameter
+			if (outputInspectorPartial == null)
+				outputInspectorPartial = udafPartial.init(partialMode,
+						parameterInspectors);
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new AlgebricksException(e);
+		}
 
-        return new AggregatuibFunctionSerializableEvaluator(parameters, types, genericUDAFName, mode, distinct,
-                rowInspector, evals, parameterInspectors, cachedParas, lazySer, cachedRowObject, udafPartial,
-                udafComplete, outputInspector, outputInspectorPartial);
-    }
+		return new AggregatuibFunctionSerializableEvaluator(parameters, types,
+				genericUDAFName, mode, distinct, rowInspector, evals,
+				parameterInspectors, cachedParas, lazySer, cachedRowObject,
+				udafPartial, udafComplete, outputInspector,
+				outputInspectorPartial);
+	}
 
-    public String toString() {
-        return "aggregation function expression evaluator factory: " + this.genericUDAFName;
-    }
+	public String toString() {
+		return "aggregation function expression evaluator factory: "
+				+ this.genericUDAFName;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java
index dc21be7..68bf408 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ColumnExpressionEvaluatorFactory.java
@@ -14,28 +14,32 @@
 

 public class ColumnExpressionEvaluatorFactory implements ICopyEvaluatorFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private ExprNodeColumnDesc expr;

+	private ExprNodeColumnDesc expr;

 

-    private Schema inputSchema;

+	private Schema inputSchema;

 

-    public ColumnExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)

-            throws AlgebricksException {

-        try {

-            expr = (ExprNodeColumnDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            throw new AlgebricksException(e.getMessage());

-        }

-        inputSchema = schema;

-    }

+	public ColumnExpressionEvaluatorFactory(ILogicalExpression expression,

+			Schema schema, IVariableTypeEnvironment env)

+			throws AlgebricksException {

+		try {

+			expr = (ExprNodeColumnDesc) ExpressionTranslator.getHiveExpression(

+					expression, env);

+		} catch (Exception e) {

+			throw new AlgebricksException(e.getMessage());

+		}

+		inputSchema = schema;

+	}

 

-    public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {

-        return new ColumnExpressionEvaluator(expr, inputSchema.toObjectInspector(), output);

-    }

+	public ICopyEvaluator createEvaluator(IDataOutputProvider output)

+			throws AlgebricksException {

+		return new ColumnExpressionEvaluator(expr,

+				inputSchema.toObjectInspector(), output);

+	}

 

-    public String toString() {

-        return "column expression evaluator factory: " + expr.toString();

-    }

+	public String toString() {

+		return "column expression evaluator factory: " + expr.toString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java
index 69e2171..e0241a1 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ConstantExpressionEvaluatorFactory.java
@@ -12,30 +12,35 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;

 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;

 

-public class ConstantExpressionEvaluatorFactory implements ICopyEvaluatorFactory {

+public class ConstantExpressionEvaluatorFactory implements

+		ICopyEvaluatorFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private ExprNodeConstantDesc expr;

+	private ExprNodeConstantDesc expr;

 

-    private Schema schema;

+	private Schema schema;

 

-    public ConstantExpressionEvaluatorFactory(ILogicalExpression expression, Schema inputSchema,

-            IVariableTypeEnvironment env) throws AlgebricksException {

-        try {

-            expr = (ExprNodeConstantDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            throw new AlgebricksException(e.getMessage());

-        }

-        schema = inputSchema;

-    }

+	public ConstantExpressionEvaluatorFactory(ILogicalExpression expression,

+			Schema inputSchema, IVariableTypeEnvironment env)

+			throws AlgebricksException {

+		try {

+			expr = (ExprNodeConstantDesc) ExpressionTranslator

+					.getHiveExpression(expression, env);

+		} catch (Exception e) {

+			throw new AlgebricksException(e.getMessage());

+		}

+		schema = inputSchema;

+	}

 

-    public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {

-        return new ConstantExpressionEvaluator(expr, schema.toObjectInspector(), output);

-    }

+	public ICopyEvaluator createEvaluator(IDataOutputProvider output)

+			throws AlgebricksException {

+		return new ConstantExpressionEvaluator(expr,

+				schema.toObjectInspector(), output);

+	}

 

-    public String toString() {

-        return "constant expression evaluator factory: " + expr.toString();

-    }

+	public String toString() {

+		return "constant expression evaluator factory: " + expr.toString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java
index eddfb9b..4b5f906 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/FieldExpressionEvaluatorFactory.java
@@ -13,28 +13,32 @@
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;

 

 public class FieldExpressionEvaluatorFactory implements ICopyEvaluatorFactory {

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private ExprNodeFieldDesc expr;

+	private ExprNodeFieldDesc expr;

 

-    private Schema inputSchema;

+	private Schema inputSchema;

 

-    public FieldExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)

-            throws AlgebricksException {

-        try {

-            expr = (ExprNodeFieldDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            throw new AlgebricksException(e.getMessage());

-        }

-        inputSchema = schema;

-    }

+	public FieldExpressionEvaluatorFactory(ILogicalExpression expression,

+			Schema schema, IVariableTypeEnvironment env)

+			throws AlgebricksException {

+		try {

+			expr = (ExprNodeFieldDesc) ExpressionTranslator.getHiveExpression(

+					expression, env);

+		} catch (Exception e) {

+			throw new AlgebricksException(e.getMessage());

+		}

+		inputSchema = schema;

+	}

 

-    public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {

-        return new FieldExpressionEvaluator(expr, inputSchema.toObjectInspector(), output);

-    }

+	public ICopyEvaluator createEvaluator(IDataOutputProvider output)

+			throws AlgebricksException {

+		return new FieldExpressionEvaluator(expr,

+				inputSchema.toObjectInspector(), output);

+	}

 

-    public String toString() {

-        return "field access expression evaluator factory: " + expr.toString();

-    }

+	public String toString() {

+		return "field access expression evaluator factory: " + expr.toString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java
index 842d96e..387ca72 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/HiveExpressionRuntimeProvider.java
@@ -34,134 +34,159 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;

 import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;

 

-public class HiveExpressionRuntimeProvider implements IExpressionRuntimeProvider {

+public class HiveExpressionRuntimeProvider implements

+		IExpressionRuntimeProvider {

 

-    public static final IExpressionRuntimeProvider INSTANCE = new HiveExpressionRuntimeProvider();

+	public static final IExpressionRuntimeProvider INSTANCE = new HiveExpressionRuntimeProvider();

 

-    @Override

-    public IAggregateEvaluatorFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new AggregateFunctionFactoryAdapter(new AggregationFunctionFactory(expr, schema, env));

-    }

+	@Override

+	public IAggregateEvaluatorFactory createAggregateFunctionFactory(

+			AggregateFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new AggregateFunctionFactoryAdapter(

+				new AggregationFunctionFactory(expr, schema, env));

+	}

 

-    @Override

-    public ICopySerializableAggregateFunctionFactory createSerializableAggregateFunctionFactory(

-            AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,

-            JobGenContext context) throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new AggregationFunctionSerializableFactory(expr, schema, env);

-    }

+	@Override

+	public ICopySerializableAggregateFunctionFactory createSerializableAggregateFunctionFactory(

+			AggregateFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new AggregationFunctionSerializableFactory(expr, schema, env);

+	}

 

-    @Override

-    public IRunningAggregateEvaluatorFactory createRunningAggregateFunctionFactory(StatefulFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public IRunningAggregateEvaluatorFactory createRunningAggregateFunctionFactory(

+			StatefulFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public IUnnestingEvaluatorFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new UnnestingFunctionFactoryAdapter(new UnnestingFunctionFactory(expr, schema, env));

-    }

+	@Override

+	public IUnnestingEvaluatorFactory createUnnestingFunctionFactory(

+			UnnestingFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new UnnestingFunctionFactoryAdapter(

+				new UnnestingFunctionFactory(expr, schema, env));

+	}

 

-    public IScalarEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,

-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {

-        switch (expr.getExpressionTag()) {

-            case VARIABLE: {

-                VariableReferenceExpression v = (VariableReferenceExpression) expr;

-                return new ScalarEvaluatorFactoryAdapter(createVariableEvaluatorFactory(v, env, inputSchemas, context));

-            }

-            case CONSTANT: {

-                ConstantExpression c = (ConstantExpression) expr;

-                return new ScalarEvaluatorFactoryAdapter(createConstantEvaluatorFactory(c, env, inputSchemas, context));

-            }

-            case FUNCTION_CALL: {

-                AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) expr;

-                FunctionIdentifier fid = fun.getFunctionIdentifier();

+	public IScalarEvaluatorFactory createEvaluatorFactory(

+			ILogicalExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		switch (expr.getExpressionTag()) {

+		case VARIABLE: {

+			VariableReferenceExpression v = (VariableReferenceExpression) expr;

+			return new ScalarEvaluatorFactoryAdapter(

+					createVariableEvaluatorFactory(v, env, inputSchemas,

+							context));

+		}

+		case CONSTANT: {

+			ConstantExpression c = (ConstantExpression) expr;

+			return new ScalarEvaluatorFactoryAdapter(

+					createConstantEvaluatorFactory(c, env, inputSchemas,

+							context));

+		}

+		case FUNCTION_CALL: {

+			AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) expr;

+			FunctionIdentifier fid = fun.getFunctionIdentifier();

 

-                if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

-                    return new ScalarEvaluatorFactoryAdapter(createFieldExpressionEvaluatorFactory(fun, env,

-                            inputSchemas, context));

-                }

+			if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

+				return new ScalarEvaluatorFactoryAdapter(

+						createFieldExpressionEvaluatorFactory(fun, env,

+								inputSchemas, context));

+			}

 

-                if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

-                    return new ScalarEvaluatorFactoryAdapter(createNullExpressionEvaluatorFactory(fun, env,

-                            inputSchemas, context));

-                }

+			if (fid.getName().equals(ExpressionConstant.FIELDACCESS)) {

+				return new ScalarEvaluatorFactoryAdapter(

+						createNullExpressionEvaluatorFactory(fun, env,

+								inputSchemas, context));

+			}

 

-                if (fun.getKind() == FunctionKind.SCALAR) {

-                    ScalarFunctionCallExpression scalar = (ScalarFunctionCallExpression) fun;

-                    return new ScalarEvaluatorFactoryAdapter(createScalarFunctionEvaluatorFactory(scalar, env,

-                            inputSchemas, context));

-                } else {

-                    throw new AlgebricksException("Cannot create evaluator for function " + fun + " of kind "

-                            + fun.getKind());

-                }

-            }

-            default: {

-                throw new IllegalStateException();

-            }

-        }

-    }

+			if (fun.getKind() == FunctionKind.SCALAR) {

+				ScalarFunctionCallExpression scalar = (ScalarFunctionCallExpression) fun;

+				return new ScalarEvaluatorFactoryAdapter(

+						createScalarFunctionEvaluatorFactory(scalar, env,

+								inputSchemas, context));

+			} else {

+				throw new AlgebricksException(

+						"Cannot create evaluator for function " + fun

+								+ " of kind " + fun.getKind());

+			}

+		}

+		default: {

+			throw new IllegalStateException();

+		}

+		}

+	}

 

-    private ICopyEvaluatorFactory createVariableEvaluatorFactory(VariableReferenceExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new ColumnExpressionEvaluatorFactory(expr, schema, env);

-    }

+	private ICopyEvaluatorFactory createVariableEvaluatorFactory(

+			VariableReferenceExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new ColumnExpressionEvaluatorFactory(expr, schema, env);

+	}

 

-    private ICopyEvaluatorFactory createScalarFunctionEvaluatorFactory(AbstractFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        List<String> names = new ArrayList<String>();

-        List<TypeInfo> types = new ArrayList<TypeInfo>();

-        for (IOperatorSchema inputSchema : inputSchemas) {

-            Schema schema = this.getSchema(inputSchema, env);

-            names.addAll(schema.getNames());

-            types.addAll(schema.getTypes());

-        }

-        Schema inputSchema = new Schema(names, types);

-        return new ScalarFunctionExpressionEvaluatorFactory(expr, inputSchema, env);

-    }

+	private ICopyEvaluatorFactory createScalarFunctionEvaluatorFactory(

+			AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		List<String> names = new ArrayList<String>();

+		List<TypeInfo> types = new ArrayList<TypeInfo>();

+		for (IOperatorSchema inputSchema : inputSchemas) {

+			Schema schema = this.getSchema(inputSchema, env);

+			names.addAll(schema.getNames());

+			types.addAll(schema.getTypes());

+		}

+		Schema inputSchema = new Schema(names, types);

+		return new ScalarFunctionExpressionEvaluatorFactory(expr, inputSchema,

+				env);

+	}

 

-    private ICopyEvaluatorFactory createFieldExpressionEvaluatorFactory(AbstractFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new FieldExpressionEvaluatorFactory(expr, schema, env);

-    }

+	private ICopyEvaluatorFactory createFieldExpressionEvaluatorFactory(

+			AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new FieldExpressionEvaluatorFactory(expr, schema, env);

+	}

 

-    private ICopyEvaluatorFactory createNullExpressionEvaluatorFactory(AbstractFunctionCallExpression expr,

-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)

-            throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new NullExpressionEvaluatorFactory(expr, schema, env);

-    }

+	private ICopyEvaluatorFactory createNullExpressionEvaluatorFactory(

+			AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new NullExpressionEvaluatorFactory(expr, schema, env);

+	}

 

-    private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr, IVariableTypeEnvironment env,

-            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {

-        Schema schema = this.getSchema(inputSchemas[0], env);

-        return new ConstantExpressionEvaluatorFactory(expr, schema, env);

-    }

+	private ICopyEvaluatorFactory createConstantEvaluatorFactory(

+			ConstantExpression expr, IVariableTypeEnvironment env,

+			IOperatorSchema[] inputSchemas, JobGenContext context)

+			throws AlgebricksException {

+		Schema schema = this.getSchema(inputSchemas[0], env);

+		return new ConstantExpressionEvaluatorFactory(expr, schema, env);

+	}

 

-    private Schema getSchema(IOperatorSchema inputSchema, IVariableTypeEnvironment env) throws AlgebricksException {

-        List<String> names = new ArrayList<String>();

-        List<TypeInfo> types = new ArrayList<TypeInfo>();

-        Iterator<LogicalVariable> variables = inputSchema.iterator();

-        while (variables.hasNext()) {

-            LogicalVariable var = variables.next();

-            names.add(var.toString());

-            types.add((TypeInfo) env.getVarType(var));

-        }

+	private Schema getSchema(IOperatorSchema inputSchema,

+			IVariableTypeEnvironment env) throws AlgebricksException {

+		List<String> names = new ArrayList<String>();

+		List<TypeInfo> types = new ArrayList<TypeInfo>();

+		Iterator<LogicalVariable> variables = inputSchema.iterator();

+		while (variables.hasNext()) {

+			LogicalVariable var = variables.next();

+			names.add(var.toString());

+			types.add((TypeInfo) env.getVarType(var));

+		}

 

-        Schema schema = new Schema(names, types);

-        return schema;

-    }

+		Schema schema = new Schema(names, types);

+		return schema;

+	}

 

 }
\ No newline at end of file
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java
index 075ed91..8f516e8 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/NullExpressionEvaluatorFactory.java
@@ -14,28 +14,32 @@
 

 public class NullExpressionEvaluatorFactory implements ICopyEvaluatorFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private ExprNodeNullDesc expr;

+	private ExprNodeNullDesc expr;

 

-    private Schema schema;

+	private Schema schema;

 

-    public NullExpressionEvaluatorFactory(ILogicalExpression expression, Schema intputSchema,

-            IVariableTypeEnvironment env) throws AlgebricksException {

-        try {

-            expr = (ExprNodeNullDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            throw new AlgebricksException(e.getMessage());

-        }

-        schema = intputSchema;

-    }

+	public NullExpressionEvaluatorFactory(ILogicalExpression expression,

+			Schema intputSchema, IVariableTypeEnvironment env)

+			throws AlgebricksException {

+		try {

+			expr = (ExprNodeNullDesc) ExpressionTranslator.getHiveExpression(

+					expression, env);

+		} catch (Exception e) {

+			throw new AlgebricksException(e.getMessage());

+		}

+		schema = intputSchema;

+	}

 

-    public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {

-        return new NullExpressionEvaluator(expr, schema.toObjectInspector(), output);

-    }

+	public ICopyEvaluator createEvaluator(IDataOutputProvider output)

+			throws AlgebricksException {

+		return new NullExpressionEvaluator(expr, schema.toObjectInspector(),

+				output);

+	}

 

-    public String toString() {

-        return "null expression evaluator factory: " + expr.toString();

-    }

+	public String toString() {

+		return "null expression evaluator factory: " + expr.toString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java
index fe7ddfb..262758e 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/ScalarFunctionExpressionEvaluatorFactory.java
@@ -14,56 +14,64 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;

 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;

 

-public class ScalarFunctionExpressionEvaluatorFactory implements ICopyEvaluatorFactory {

+public class ScalarFunctionExpressionEvaluatorFactory implements

+		ICopyEvaluatorFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private transient ExprNodeGenericFuncDesc expr;

+	private transient ExprNodeGenericFuncDesc expr;

 

-    private String exprSerialization;

+	private String exprSerialization;

 

-    private Schema inputSchema;

+	private Schema inputSchema;

 

-    private transient Configuration config;

+	private transient Configuration config;

 

-    public ScalarFunctionExpressionEvaluatorFactory(ILogicalExpression expression, Schema schema,

-            IVariableTypeEnvironment env) throws AlgebricksException {

-        try {

-            expr = (ExprNodeGenericFuncDesc) ExpressionTranslator.getHiveExpression(expression, env);

+	public ScalarFunctionExpressionEvaluatorFactory(

+			ILogicalExpression expression, Schema schema,

+			IVariableTypeEnvironment env) throws AlgebricksException {

+		try {

+			expr = (ExprNodeGenericFuncDesc) ExpressionTranslator

+					.getHiveExpression(expression, env);

 

-            exprSerialization = Utilities.serializeExpression(expr);

+			exprSerialization = Utilities.serializeExpression(expr);

 

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new AlgebricksException(e.getMessage());

-        }

-        inputSchema = schema;

-    }

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new AlgebricksException(e.getMessage());

+		}

+		inputSchema = schema;

+	}

 

-    public synchronized ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {

-        if (expr == null) {

-            configClassLoader();

-            expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(exprSerialization, config);

-        }

+	public synchronized ICopyEvaluator createEvaluator(

+			IDataOutputProvider output) throws AlgebricksException {

+		if (expr == null) {

+			configClassLoader();

+			expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(

+					exprSerialization, config);

+		}

 

-        ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc) expr.clone();

-        return new FunctionExpressionEvaluator(funcDesc, inputSchema.toObjectInspector(), output);

-    }

+		ExprNodeGenericFuncDesc funcDesc = (ExprNodeGenericFuncDesc) expr

+				.clone();

+		return new FunctionExpressionEvaluator(funcDesc,

+				inputSchema.toObjectInspector(), output);

+	}

 

-    private void configClassLoader() {

-        config = new Configuration();

-        ClassLoader loader = this.getClass().getClassLoader();

-        config.setClassLoader(loader);

-        Thread.currentThread().setContextClassLoader(loader);

-    }

+	private void configClassLoader() {

+		config = new Configuration();

+		ClassLoader loader = this.getClass().getClassLoader();

+		config.setClassLoader(loader);

+		Thread.currentThread().setContextClassLoader(loader);

+	}

 

-    public String toString() {

-        if (expr == null){

-            configClassLoader();

-            expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(exprSerialization, new Configuration());

-        }

+	public String toString() {

+		if (expr == null) {

+			configClassLoader();

+			expr = (ExprNodeGenericFuncDesc) Utilities.deserializeExpression(

+					exprSerialization, new Configuration());

+		}

 

-        return "function expression evaluator factory: " + expr.getExprString();

-    }

+		return "function expression evaluator factory: " + expr.getExprString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java
index 4657ae1..1d77737 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/evaluator/UnnestingFunctionFactory.java
@@ -14,27 +14,31 @@
 

 public class UnnestingFunctionFactory implements ICopyUnnestingFunctionFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private UDTFDesc expr;

+	private UDTFDesc expr;

 

-    private Schema inputSchema;

+	private Schema inputSchema;

 

-    private int[] columns;

+	private int[] columns;

 

-    public UnnestingFunctionFactory(ILogicalExpression expression, Schema schema, IVariableTypeEnvironment env)

-            throws AlgebricksException {

-        try {

-            expr = (UDTFDesc) ExpressionTranslator.getHiveExpression(expression, env);

-        } catch (Exception e) {

-            throw new AlgebricksException(e.getMessage());

-        }

-        inputSchema = schema;

-    }

+	public UnnestingFunctionFactory(ILogicalExpression expression,

+			Schema schema, IVariableTypeEnvironment env)

+			throws AlgebricksException {

+		try {

+			expr = (UDTFDesc) ExpressionTranslator.getHiveExpression(

+					expression, env);

+		} catch (Exception e) {

+			throw new AlgebricksException(e.getMessage());

+		}

+		inputSchema = schema;

+	}

 

-    @Override

-    public ICopyUnnestingFunction createUnnestingFunction(IDataOutputProvider provider) throws AlgebricksException {

-        return new UDTFFunctionEvaluator(expr, inputSchema, columns, provider.getDataOutput());

-    }

+	@Override

+	public ICopyUnnestingFunction createUnnestingFunction(

+			IDataOutputProvider provider) throws AlgebricksException {

+		return new UDTFFunctionEvaluator(expr, inputSchema, columns,

+				provider.getDataOutput());

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java
index b636009..fc302e1 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveDoubleBinaryHashFunctionFactory.java
@@ -4,26 +4,28 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveDoubleBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveDoubleBinaryHashFunctionFactory implements

+		IBinaryHashFunctionFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveDoubleBinaryHashFunctionFactory INSTANCE = new HiveDoubleBinaryHashFunctionFactory();

+	public static HiveDoubleBinaryHashFunctionFactory INSTANCE = new HiveDoubleBinaryHashFunctionFactory();

 

-    private HiveDoubleBinaryHashFunctionFactory() {

-    }

+	private HiveDoubleBinaryHashFunctionFactory() {

+	}

 

-    @Override

-    public IBinaryHashFunction createBinaryHashFunction() {

-        // TODO Auto-generated method stub

-        return new IBinaryHashFunction() {

-            private Double value;

+	@Override

+	public IBinaryHashFunction createBinaryHashFunction() {

+		// TODO Auto-generated method stub

+		return new IBinaryHashFunction() {

+			private Double value;

 

-            @Override

-            public int hash(byte[] bytes, int offset, int length) {

-                value = Double.longBitsToDouble(LazyUtils.byteArrayToLong(bytes, offset));

-                return value.hashCode();

-            }

-        };

-    }

+			@Override

+			public int hash(byte[] bytes, int offset, int length) {

+				value = Double.longBitsToDouble(LazyUtils.byteArrayToLong(

+						bytes, offset));

+				return value.hashCode();

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java
index 90e6ce4..e1a9994 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveIntegerBinaryHashFunctionFactory.java
@@ -5,29 +5,31 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveIntegerBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveIntegerBinaryHashFunctionFactory implements

+		IBinaryHashFunctionFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static IBinaryHashFunctionFactory INSTANCE = new HiveIntegerBinaryHashFunctionFactory();

+	public static IBinaryHashFunctionFactory INSTANCE = new HiveIntegerBinaryHashFunctionFactory();

 

-    private HiveIntegerBinaryHashFunctionFactory() {

-    }

+	private HiveIntegerBinaryHashFunctionFactory() {

+	}

 

-    @Override

-    public IBinaryHashFunction createBinaryHashFunction() {

+	@Override

+	public IBinaryHashFunction createBinaryHashFunction() {

 

-        return new IBinaryHashFunction() {

-            private VInt value = new VInt();

+		return new IBinaryHashFunction() {

+			private VInt value = new VInt();

 

-            @Override

-            public int hash(byte[] bytes, int offset, int length) {

-                LazyUtils.readVInt(bytes, offset, value);

-                if (value.length != length)

-                    throw new IllegalArgumentException("length mismatch in int hash function actual: " + length

-                            + " expected " + value.length);

-                return value.value;

-            }

-        };

-    }

+			@Override

+			public int hash(byte[] bytes, int offset, int length) {

+				LazyUtils.readVInt(bytes, offset, value);

+				if (value.length != length)

+					throw new IllegalArgumentException(

+							"length mismatch in int hash function actual: "

+									+ length + " expected " + value.length);

+				return value.value;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java
index 1b61f67..6f7c6f2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveLongBinaryHashFunctionFactory.java
@@ -5,26 +5,27 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveLongBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveLongBinaryHashFunctionFactory implements

+		IBinaryHashFunctionFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static IBinaryHashFunctionFactory INSTANCE = new HiveLongBinaryHashFunctionFactory();

+	public static IBinaryHashFunctionFactory INSTANCE = new HiveLongBinaryHashFunctionFactory();

 

-    private HiveLongBinaryHashFunctionFactory() {

-    }

+	private HiveLongBinaryHashFunctionFactory() {

+	}

 

-    @Override

-    public IBinaryHashFunction createBinaryHashFunction() {

+	@Override

+	public IBinaryHashFunction createBinaryHashFunction() {

 

-        return new IBinaryHashFunction() {

-            private VLong value = new VLong();

+		return new IBinaryHashFunction() {

+			private VLong value = new VLong();

 

-            @Override

-            public int hash(byte[] bytes, int offset, int length) {

-                LazyUtils.readVLong(bytes, offset, value);

-                return (int) value.value;

-            }

-        };

-    }

+			@Override

+			public int hash(byte[] bytes, int offset, int length) {

+				LazyUtils.readVLong(bytes, offset, value);

+				return (int) value.value;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java
index f2b7b44..e03dde0 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveRawBinaryHashFunctionFactory.java
@@ -3,29 +3,30 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveRawBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveRawBinaryHashFunctionFactory implements

+		IBinaryHashFunctionFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static IBinaryHashFunctionFactory INSTANCE = new HiveRawBinaryHashFunctionFactory();

+	public static IBinaryHashFunctionFactory INSTANCE = new HiveRawBinaryHashFunctionFactory();

 

-    private HiveRawBinaryHashFunctionFactory() {

+	private HiveRawBinaryHashFunctionFactory() {

 

-    }

+	}

 

-    @Override

-    public IBinaryHashFunction createBinaryHashFunction() {

+	@Override

+	public IBinaryHashFunction createBinaryHashFunction() {

 

-        return new IBinaryHashFunction() {

+		return new IBinaryHashFunction() {

 

-            @Override

-            public int hash(byte[] bytes, int offset, int length) {

-                int value = 1;

-                int end = offset + length;

-                for (int i = offset; i < end; i++)

-                    value = value * 31 + (int) bytes[i];

-                return value;

-            }

-        };

-    }

+			@Override

+			public int hash(byte[] bytes, int offset, int length) {

+				int value = 1;

+				int end = offset + length;

+				for (int i = offset; i < end; i++)

+					value = value * 31 + (int) bytes[i];

+				return value;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java
index a9cf6fd..055c077 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/HiveStingBinaryHashFunctionFactory.java
@@ -5,37 +5,41 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveStingBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {

-    private static final long serialVersionUID = 1L;

+public class HiveStingBinaryHashFunctionFactory implements

+		IBinaryHashFunctionFactory {

+	private static final long serialVersionUID = 1L;

 

-    public static HiveStingBinaryHashFunctionFactory INSTANCE = new HiveStingBinaryHashFunctionFactory();

+	public static HiveStingBinaryHashFunctionFactory INSTANCE = new HiveStingBinaryHashFunctionFactory();

 

-    private HiveStingBinaryHashFunctionFactory() {

-    }

+	private HiveStingBinaryHashFunctionFactory() {

+	}

 

-    @Override

-    public IBinaryHashFunction createBinaryHashFunction() {

-        // TODO Auto-generated method stub

-        return new IBinaryHashFunction() {

-            private VInt len = new VInt();

+	@Override

+	public IBinaryHashFunction createBinaryHashFunction() {

+		// TODO Auto-generated method stub

+		return new IBinaryHashFunction() {

+			private VInt len = new VInt();

 

-            @Override

-            public int hash(byte[] bytes, int offset, int length) {

-                LazyUtils.readVInt(bytes, offset, len);

-                if (len.value + len.length != length)

-                    throw new IllegalStateException("parse string: length mismatch, expected "

-                            + (len.value + len.length) + " but get " + length);

-                return hashBytes(bytes, offset + len.length, length - len.length);

-            }

+			@Override

+			public int hash(byte[] bytes, int offset, int length) {

+				LazyUtils.readVInt(bytes, offset, len);

+				if (len.value + len.length != length)

+					throw new IllegalStateException(

+							"parse string: length mismatch, expected "

+									+ (len.value + len.length) + " but get "

+									+ length);

+				return hashBytes(bytes, offset + len.length, length

+						- len.length);

+			}

 

-            public int hashBytes(byte[] bytes, int offset, int length) {

-                int value = 1;

-                int end = offset + length;

-                for (int i = offset; i < end; i++)

-                    value = value * 31 + (int) bytes[i];

-                return value;

-            }

-        };

-    }

+			public int hashBytes(byte[] bytes, int offset, int length) {

+				int value = 1;

+				int end = offset + length;

+				for (int i = offset; i < end; i++)

+					value = value * 31 + (int) bytes[i];

+				return value;

+			}

+		};

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
new file mode 100644
index 0000000..760a614
--- /dev/null
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
@@ -0,0 +1,63 @@
+package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+
+public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
+
+    public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
+
+    private static final long serialVersionUID = 1L;
+
+    private MurmurHash3BinaryHashFunctionFamily() {
+    }
+
+    private static final int C1 = 0xcc9e2d51;
+    private static final int C2 = 0x1b873593;
+    private static final int C3 = 5;
+    private static final int C4 = 0xe6546b64;
+    private static final int C5 = 0x85ebca6b;
+    private static final int C6 = 0xc2b2ae35;
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction(final int seed) {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int h = seed;
+                int p = offset;
+                int remain = length;
+                while (remain >= 4) {
+                    int k = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8) | ((bytes[p + 2] & 0xff) << 16)
+                            | ((bytes[p + 3] & 0xff) << 24);
+                    k *= C1;
+                    k = Integer.rotateLeft(k, 15);
+                    k *= C2;
+                    h ^= k;
+                    h = Integer.rotateLeft(h, 13);
+                    h = h * C3 + C4;
+                    p += 4;
+                    remain -= 4;
+                }
+                if (remain > 0) {
+                    int k = 0;
+                    for (int i = 0; remain > 0; i += 8) {
+                        k ^= (bytes[p++] & 0xff) << i;
+                        remain--;
+                    }
+                    k *= C1;
+                    k = Integer.rotateLeft(k, 15);
+                    k *= C2;
+                    h ^= k;
+                }
+                h ^= length;
+                h ^= (h >>> 16);
+                h *= C5;
+                h ^= (h >>> 13);
+                h *= C6;
+                h ^= (h >>> 16);
+                return h;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java
index 6ac012f..5f03962 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleAscNormalizedKeyComputerFactory.java
@@ -4,21 +4,22 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveDoubleAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveDoubleAscNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
 
-        return new INormalizedKeyComputer() {
+		return new INormalizedKeyComputer() {
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                int header = LazyUtils.byteArrayToInt(bytes, start);
-                long unsignedValue = (long) header;
-                return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
-            }
-        };
-    }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				int header = LazyUtils.byteArrayToInt(bytes, start);
+				long unsignedValue = (long) header;
+				return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java
index 3044109..e4587a2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveDoubleDescNormalizedKeyComputerFactory.java
@@ -3,22 +3,24 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveDoubleDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveDoubleDescNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
-    private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveDoubleAscNormalizedKeyComputerFactory();
+	private static final long serialVersionUID = 1L;
+	private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveDoubleAscNormalizedKeyComputerFactory();
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
-        return new INormalizedKeyComputer() {
-            private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory.createNormalizedKeyComputer();
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
+		return new INormalizedKeyComputer() {
+			private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory
+					.createNormalizedKeyComputer();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                int nk = nmkComputer.normalize(bytes, start, length);
-                return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
-            }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				int nk = nmkComputer.normalize(bytes, start, length);
+				return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
+			}
 
-        };
-    }
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java
index a1d4d48..2ff390a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerAscNormalizedKeyComputerFactory.java
@@ -5,25 +5,27 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveIntegerAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveIntegerAscNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
 
-        return new INormalizedKeyComputer() {
-            private VInt vint = new VInt();
+		return new INormalizedKeyComputer() {
+			private VInt vint = new VInt();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                LazyUtils.readVInt(bytes, start, vint);
-                if (vint.length != length)
-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "
-                            + vint.length + " expected " + length);
-                long unsignedValue = (long) vint.value;
-                return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
-            }
-        };
-    }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				LazyUtils.readVInt(bytes, start, vint);
+				if (vint.length != length)
+					throw new IllegalArgumentException(
+							"length mismatch in int comparator function actual: "
+									+ vint.length + " expected " + length);
+				long unsignedValue = (long) vint.value;
+				return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java
index 39d4a4b..8eff1f8 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveIntegerDescNormalizedKeyComputerFactory.java
@@ -5,25 +5,27 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveIntegerDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveIntegerDescNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
 
-        return new INormalizedKeyComputer() {
-            private VInt vint = new VInt();
+		return new INormalizedKeyComputer() {
+			private VInt vint = new VInt();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                LazyUtils.readVInt(bytes, start, vint);
-                if (vint.length != length)
-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "
-                            + vint.length + " expected " + length);
-                long unsignedValue = (long) vint.value;
-                return (int) ((long)0xffffffff - unsignedValue);
-            }
-        };
-    }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				LazyUtils.readVInt(bytes, start, vint);
+				if (vint.length != length)
+					throw new IllegalArgumentException(
+							"length mismatch in int comparator function actual: "
+									+ vint.length + " expected " + length);
+				long unsignedValue = (long) vint.value;
+				return (int) ((long) 0xffffffff - unsignedValue);
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java
index 079e069..768eec2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongAscNormalizedKeyComputerFactory.java
@@ -5,59 +5,61 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveLongAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveLongAscNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
 
-        return new INormalizedKeyComputer() {
-            private static final int POSTIVE_LONG_MASK = (3 << 30);
-            private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
-            private static final int NEGATIVE_LONG_MASK = (0 << 30);
-            private VLong vlong = new VLong();
+		return new INormalizedKeyComputer() {
+			private static final int POSTIVE_LONG_MASK = (3 << 30);
+			private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+			private static final int NEGATIVE_LONG_MASK = (0 << 30);
+			private VLong vlong = new VLong();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                LazyUtils.readVLong(bytes, start, vlong);
-                if (vlong.length != length)
-                    throw new IllegalArgumentException("length mismatch in int comparator function actual: "
-                            + vlong.length + " expected " + length);
-                long value = (long) vlong.value;
-                int highValue = (int) (value >> 32);
-                if (highValue > 0) {
-                    /**
-                     * larger than Integer.MAX
-                     */
-                    int highNmk = getKey(highValue);
-                    highNmk >>= 2;
-                    highNmk |= POSTIVE_LONG_MASK;
-                    return highNmk;
-                } else if (highValue == 0) {
-                    /**
-                     * smaller than Integer.MAX but >=0
-                     */
-                    int lowNmk = (int) value;
-                    lowNmk >>= 2;
-                    lowNmk |= NON_NEGATIVE_INT_MASK;
-                    return lowNmk;
-                } else {
-                    /**
-                     * less than 0; TODO: have not optimized for that
-                     */
-                    int highNmk = getKey(highValue);
-                    highNmk >>= 2;
-                    highNmk |= NEGATIVE_LONG_MASK;
-                    return highNmk;
-                }
-            }
-            
-            private int getKey(int value) {
-                long unsignedFirstValue = (long) value;
-                int nmk = (int) ((unsignedFirstValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
-                return nmk;
-            }
-        };
-    }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				LazyUtils.readVLong(bytes, start, vlong);
+				if (vlong.length != length)
+					throw new IllegalArgumentException(
+							"length mismatch in int comparator function actual: "
+									+ vlong.length + " expected " + length);
+				long value = (long) vlong.value;
+				int highValue = (int) (value >> 32);
+				if (highValue > 0) {
+					/**
+					 * larger than Integer.MAX
+					 */
+					int highNmk = getKey(highValue);
+					highNmk >>= 2;
+					highNmk |= POSTIVE_LONG_MASK;
+					return highNmk;
+				} else if (highValue == 0) {
+					/**
+					 * smaller than Integer.MAX but >=0
+					 */
+					int lowNmk = (int) value;
+					lowNmk >>= 2;
+					lowNmk |= NON_NEGATIVE_INT_MASK;
+					return lowNmk;
+				} else {
+					/**
+					 * less than 0; TODO: have not optimized for that
+					 */
+					int highNmk = getKey(highValue);
+					highNmk >>= 2;
+					highNmk |= NEGATIVE_LONG_MASK;
+					return highNmk;
+				}
+			}
+
+			private int getKey(int value) {
+				long unsignedFirstValue = (long) value;
+				int nmk = (int) ((unsignedFirstValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+				return nmk;
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java
index cc5661b..20ae56a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveLongDescNormalizedKeyComputerFactory.java
@@ -3,23 +3,25 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveLongDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveLongDescNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
-    private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveIntegerAscNormalizedKeyComputerFactory();
+	private static final long serialVersionUID = 1L;
+	private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new HiveIntegerAscNormalizedKeyComputerFactory();
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
-        return new INormalizedKeyComputer() {
-            private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory.createNormalizedKeyComputer();
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
+		return new INormalizedKeyComputer() {
+			private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory
+					.createNormalizedKeyComputer();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                int nk = nmkComputer.normalize(bytes, start, length);
-                return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
-            }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				int nk = nmkComputer.normalize(bytes, start, length);
+				return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
+			}
 
-        };
-    }
+		};
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java
index d0429d6..b16ccba 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringAscNormalizedKeyComputerFactory.java
@@ -6,35 +6,39 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class HiveStringAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveStringAscNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
 
-        return new INormalizedKeyComputer() {
-            private VInt len = new VInt();
+		return new INormalizedKeyComputer() {
+			private VInt len = new VInt();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                LazyUtils.readVInt(bytes, start, len);
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				LazyUtils.readVInt(bytes, start, len);
 
-                if (len.value + len.length != length)
-                    throw new IllegalStateException("parse string: length mismatch, expected "
-                            + (len.value + len.length) + " but get " + length);
-                int nk = 0;
-                int offset = start + len.length;
-                for (int i = 0; i < 2; ++i) {
-                    nk <<= 16;
-                    if (i < len.value) {
-                        char character = UTF8StringPointable.charAt(bytes, offset);
-                        nk += ((int) character) & 0xffff;
-                        offset += UTF8StringPointable.charSize(bytes, offset);
-                    }
-                }
-                return nk;
-            }
-        };
-    }
+				if (len.value + len.length != length)
+					throw new IllegalStateException(
+							"parse string: length mismatch, expected "
+									+ (len.value + len.length) + " but get "
+									+ length);
+				int nk = 0;
+				int offset = start + len.length;
+				for (int i = 0; i < 2; ++i) {
+					nk <<= 16;
+					if (i < len.value) {
+						char character = UTF8StringPointable.charAt(bytes,
+								offset);
+						nk += ((int) character) & 0xffff;
+						offset += UTF8StringPointable.charSize(bytes, offset);
+					}
+				}
+				return nk;
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java
index 15b2d27..e8978c6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/normalize/HiveStringDescNormalizedKeyComputerFactory.java
@@ -6,32 +6,35 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class HiveStringDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+public class HiveStringDescNormalizedKeyComputerFactory implements
+		INormalizedKeyComputerFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
-        return new INormalizedKeyComputer() {
-            private VInt len = new VInt();
+	@Override
+	public INormalizedKeyComputer createNormalizedKeyComputer() {
+		return new INormalizedKeyComputer() {
+			private VInt len = new VInt();
 
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                LazyUtils.readVInt(bytes, start, len);
-                if (len.value + len.length != length)
-                    throw new IllegalStateException("parse string: length mismatch, expected "
-                            + (len.value + len.length) + " but get " + length);
-                int nk = 0;
-                int offset = start + len.length;
-                for (int i = 0; i < 2; ++i) {
-                    nk <<= 16;
-                    if (i < len.value) {
-                        nk += ((int) UTF8StringPointable.charAt(bytes, offset)) & 0xffff;
-                        offset += UTF8StringPointable.charSize(bytes, offset);
-                    }
-                }
-                return (int) ((long) 0xffffffff - (long) nk);
-            }
-        };
-    }
+			@Override
+			public int normalize(byte[] bytes, int start, int length) {
+				LazyUtils.readVInt(bytes, start, len);
+				if (len.value + len.length != length)
+					throw new IllegalStateException(
+							"parse string: length mismatch, expected "
+									+ (len.value + len.length) + " but get "
+									+ length);
+				int nk = 0;
+				int offset = start + len.length;
+				for (int i = 0; i < 2; ++i) {
+					nk <<= 16;
+					if (i < len.value) {
+						nk += ((int) UTF8StringPointable.charAt(bytes, offset)) & 0xffff;
+						offset += UTF8StringPointable.charSize(bytes, offset);
+					}
+				}
+				return (int) ((long) 0xffffffff - (long) nk);
+			}
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java
index 590bd61..91d08c6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/factory/nullwriter/HiveNullWriterFactory.java
@@ -8,21 +8,21 @@
 
 public class HiveNullWriterFactory implements INullWriterFactory {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    public static HiveNullWriterFactory INSTANCE = new HiveNullWriterFactory();
+	public static HiveNullWriterFactory INSTANCE = new HiveNullWriterFactory();
 
-    @Override
-    public INullWriter createNullWriter() {
-        return new HiveNullWriter();
-    }
+	@Override
+	public INullWriter createNullWriter() {
+		return new HiveNullWriter();
+	}
 }
 
 class HiveNullWriter implements INullWriter {
 
-    @Override
-    public void writeNull(DataOutput out) throws HyracksDataException {
-        // do nothing
-    }
+	@Override
+	public void writeNull(DataOutput out) throws HyracksDataException {
+		// do nothing
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java
index 677e20e..3d2b141 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspector.java
@@ -4,16 +4,17 @@
 

 public class HiveBinaryBooleanInspector implements IBinaryBooleanInspector {

 

-    HiveBinaryBooleanInspector() {

-    }

+	HiveBinaryBooleanInspector() {

+	}

 

-    @Override

-    public boolean getBooleanValue(byte[] bytes, int offset, int length) {

-        if (length == 0)

-            return false;

-        if (length != 1)

-            throw new IllegalStateException("boolean field error: with length " + length);

-        return bytes[0] == 1;

-    }

+	@Override

+	public boolean getBooleanValue(byte[] bytes, int offset, int length) {

+		if (length == 0)

+			return false;

+		if (length != 1)

+			throw new IllegalStateException("boolean field error: with length "

+					+ length);

+		return bytes[0] == 1;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java
index 22a6065..86afbee 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryBooleanInspectorFactory.java
@@ -4,17 +4,19 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
-public class HiveBinaryBooleanInspectorFactory implements IBinaryBooleanInspectorFactory {
-    private static final long serialVersionUID = 1L;
-    public static HiveBinaryBooleanInspectorFactory INSTANCE = new HiveBinaryBooleanInspectorFactory();
+public class HiveBinaryBooleanInspectorFactory implements
+		IBinaryBooleanInspectorFactory {
+	private static final long serialVersionUID = 1L;
+	public static HiveBinaryBooleanInspectorFactory INSTANCE = new HiveBinaryBooleanInspectorFactory();
 
-    private HiveBinaryBooleanInspectorFactory() {
+	private HiveBinaryBooleanInspectorFactory() {
 
-    }
+	}
 
-    @Override
-    public IBinaryBooleanInspector createBinaryBooleanInspector(IHyracksTaskContext arg0) {
-        return new HiveBinaryBooleanInspector();
-    }
+	@Override
+	public IBinaryBooleanInspector createBinaryBooleanInspector(
+			IHyracksTaskContext arg0) {
+		return new HiveBinaryBooleanInspector();
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java
index 555afee..e82e501 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspector.java
@@ -5,18 +5,19 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;

 

 public class HiveBinaryIntegerInspector implements IBinaryIntegerInspector {

-    private VInt value = new VInt();

+	private VInt value = new VInt();

 

-    HiveBinaryIntegerInspector() {

-    }

+	HiveBinaryIntegerInspector() {

+	}

 

-    @Override

-    public int getIntegerValue(byte[] bytes, int offset, int length) {

-        LazyUtils.readVInt(bytes, offset, value);

-        if (value.length != length)

-            throw new IllegalArgumentException("length mismatch in int hash function actual: " + length + " expected "

-                    + value.length);

-        return value.value;

-    }

+	@Override

+	public int getIntegerValue(byte[] bytes, int offset, int length) {

+		LazyUtils.readVInt(bytes, offset, value);

+		if (value.length != length)

+			throw new IllegalArgumentException(

+					"length mismatch in int hash function actual: " + length

+							+ " expected " + value.length);

+		return value.value;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java
index bb93a60..b44e610 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/inspector/HiveBinaryIntegerInspectorFactory.java
@@ -4,17 +4,19 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
-public class HiveBinaryIntegerInspectorFactory implements IBinaryIntegerInspectorFactory {
-    private static final long serialVersionUID = 1L;
-    public static HiveBinaryIntegerInspectorFactory INSTANCE = new HiveBinaryIntegerInspectorFactory();
+public class HiveBinaryIntegerInspectorFactory implements
+		IBinaryIntegerInspectorFactory {
+	private static final long serialVersionUID = 1L;
+	public static HiveBinaryIntegerInspectorFactory INSTANCE = new HiveBinaryIntegerInspectorFactory();
 
-    private HiveBinaryIntegerInspectorFactory() {
+	private HiveBinaryIntegerInspectorFactory() {
 
-    }
+	}
 
-    @Override
-    public IBinaryIntegerInspector createBinaryIntegerInspector(IHyracksTaskContext arg0) {
-        return new HiveBinaryIntegerInspector();
-    }
+	@Override
+	public IBinaryIntegerInspector createBinaryIntegerInspector(
+			IHyracksTaskContext arg0) {
+		return new HiveBinaryIntegerInspector();
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java
index bdced12..8f559e2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveConnectorPolicyAssignmentPolicy.java
@@ -10,56 +10,58 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 
-public class HiveConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
-    public enum Policy {
-        PIPELINING, SEND_SIDE_MAT_PIPELINING, SEND_SIDE_MAT_BLOCKING, SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING;
-    };
+public class HiveConnectorPolicyAssignmentPolicy implements
+		IConnectorPolicyAssignmentPolicy {
+	public enum Policy {
+		PIPELINING, SEND_SIDE_MAT_PIPELINING, SEND_SIDE_MAT_BLOCKING, SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING;
+	};
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    private final IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy();
-    private final IConnectorPolicy sendSideMatPipeliningPolicy = new SendSideMaterializedPipeliningConnectorPolicy();
-    private final IConnectorPolicy sendSideMatBlockingPolicy = new SendSideMaterializedBlockingConnectorPolicy();
-    private final IConnectorPolicy sendSideMatReceiveSideMatBlockingPolicy = new SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy();
-    private final Policy policy;
+	private final IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy();
+	private final IConnectorPolicy sendSideMatPipeliningPolicy = new SendSideMaterializedPipeliningConnectorPolicy();
+	private final IConnectorPolicy sendSideMatBlockingPolicy = new SendSideMaterializedBlockingConnectorPolicy();
+	private final IConnectorPolicy sendSideMatReceiveSideMatBlockingPolicy = new SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy();
+	private final Policy policy;
 
-    public HiveConnectorPolicyAssignmentPolicy(Policy policy) {
-        this.policy = policy;
-    }
+	public HiveConnectorPolicyAssignmentPolicy(Policy policy) {
+		this.policy = policy;
+	}
 
-    @Override
-    public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
-            int[] fanouts) {
-        if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
-            // avoid deadlocks
-            switch (policy) {
-                case PIPELINING:
-                case SEND_SIDE_MAT_PIPELINING:
-                    return sendSideMatPipeliningPolicy;
-                case SEND_SIDE_MAT_BLOCKING:
-                    return sendSideMatBlockingPolicy;
-                case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
-                    return sendSideMatReceiveSideMatBlockingPolicy;
-                default:
-                    return sendSideMatPipeliningPolicy;
-            }
-        } else if (c instanceof MToNPartitioningConnectorDescriptor) {
-            // support different repartitioning policies
-            switch (policy) {
-                case PIPELINING:
-                    return pipeliningPolicy;
-                case SEND_SIDE_MAT_PIPELINING:
-                    return sendSideMatPipeliningPolicy;
-                case SEND_SIDE_MAT_BLOCKING:
-                    return sendSideMatBlockingPolicy;
-                case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
-                    return sendSideMatReceiveSideMatBlockingPolicy;
-                default:
-                    return pipeliningPolicy;
-            }
-        } else {
-            // pipelining for other connectors
-            return pipeliningPolicy;
-        }
-    }
+	@Override
+	public IConnectorPolicy getConnectorPolicyAssignment(
+			IConnectorDescriptor c, int nProducers, int nConsumers,
+			int[] fanouts) {
+		if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
+			// avoid deadlocks
+			switch (policy) {
+			case PIPELINING:
+			case SEND_SIDE_MAT_PIPELINING:
+				return sendSideMatPipeliningPolicy;
+			case SEND_SIDE_MAT_BLOCKING:
+				return sendSideMatBlockingPolicy;
+			case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
+				return sendSideMatReceiveSideMatBlockingPolicy;
+			default:
+				return sendSideMatPipeliningPolicy;
+			}
+		} else if (c instanceof MToNPartitioningConnectorDescriptor) {
+			// support different repartitioning policies
+			switch (policy) {
+			case PIPELINING:
+				return pipeliningPolicy;
+			case SEND_SIDE_MAT_PIPELINING:
+				return sendSideMatPipeliningPolicy;
+			case SEND_SIDE_MAT_BLOCKING:
+				return sendSideMatBlockingPolicy;
+			case SEND_SIDE_MAT_RECEIVE_SIDE_MAT_BLOCKING:
+				return sendSideMatReceiveSideMatBlockingPolicy;
+			default:
+				return pipeliningPolicy;
+			}
+		} else {
+			// pipelining for other connectors
+			return pipeliningPolicy;
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java
index ccc2e6c..e4fbca5 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSink.java
@@ -6,27 +6,27 @@
 

 public class HiveDataSink implements IDataSink {

 

-    private Object[] schema;

+	private Object[] schema;

 

-    private Object fsOperator;

+	private Object fsOperator;

 

-    public HiveDataSink(Object sink, Object[] sourceSchema) {

-        schema = sourceSchema;

-        fsOperator = sink;

-    }

+	public HiveDataSink(Object sink, Object[] sourceSchema) {

+		schema = sourceSchema;

+		fsOperator = sink;

+	}

 

-    @Override

-    public Object getId() {

-        return fsOperator;

-    }

+	@Override

+	public Object getId() {

+		return fsOperator;

+	}

 

-    @Override

-    public Object[] getSchemaTypes() {

-        return schema;

-    }

+	@Override

+	public Object[] getSchemaTypes() {

+		return schema;

+	}

 

-    public IPartitioningProperty getPartitioningProperty() {

-        return new RandomPartitioningProperty(new HiveDomain());

-    }

+	public IPartitioningProperty getPartitioningProperty() {

+		return new RandomPartitioningProperty(new HiveDomain());

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java
index 67b743b..edff056 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSource.java
@@ -11,37 +11,38 @@
 

 public class HiveDataSource<P> implements IDataSource<P> {

 

-    private P source;

+	private P source;

 

-    private Object[] schema;

+	private Object[] schema;

 

-    public HiveDataSource(P dataSource, Object[] sourceSchema) {

-        source = dataSource;

-        schema = sourceSchema;

-    }

+	public HiveDataSource(P dataSource, Object[] sourceSchema) {

+		source = dataSource;

+		schema = sourceSchema;

+	}

 

-    @Override

-    public P getId() {

-        return source;

-    }

+	@Override

+	public P getId() {

+		return source;

+	}

 

-    @Override

-    public Object[] getSchemaTypes() {

-        return schema;

-    }

+	@Override

+	public Object[] getSchemaTypes() {

+		return schema;

+	}

 

-    @Override

-    public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList) {

-    }

+	@Override

+	public void computeFDs(List<LogicalVariable> scanVariables,

+			List<FunctionalDependency> fdList) {

+	}

 

-    @Override

-    public IDataSourcePropertiesProvider getPropertiesProvider() {

-        return new HiveDataSourcePartitioningProvider();

-    }

+	@Override

+	public IDataSourcePropertiesProvider getPropertiesProvider() {

+		return new HiveDataSourcePartitioningProvider();

+	}

 

-    @Override

-    public String toString() {

-        PartitionDesc desc = (PartitionDesc) source;

-        return desc.getTableName();

-    }

+	@Override

+	public String toString() {

+		PartitionDesc desc = (PartitionDesc) source;

+		return desc.getTableName();

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java
index bb9c4ce..08dd684 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDataSourcePartitioningProvider.java
@@ -11,13 +11,16 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;

 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;

 

-public class HiveDataSourcePartitioningProvider implements IDataSourcePropertiesProvider {

+public class HiveDataSourcePartitioningProvider implements

+		IDataSourcePropertiesProvider {

 

-    @Override

-    public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {

-        IPartitioningProperty property = new RandomPartitioningProperty(new HiveDomain());

-        IPhysicalPropertiesVector vector = new StructuralPropertiesVector(property,

-                new LinkedList<ILocalStructuralProperty>());

-        return vector;

-    }

+	@Override

+	public IPhysicalPropertiesVector computePropertiesVector(

+			List<LogicalVariable> scanVariables) {

+		IPartitioningProperty property = new RandomPartitioningProperty(

+				new HiveDomain());

+		IPhysicalPropertiesVector vector = new StructuralPropertiesVector(

+				property, new LinkedList<ILocalStructuralProperty>());

+		return vector;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java
index 8b1d3b5..0af253a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveDomain.java
@@ -4,14 +4,14 @@
 

 public class HiveDomain implements INodeDomain {

 

-    @Override

-    public boolean sameAs(INodeDomain domain) {

-        return true;

-    }

+	@Override

+	public boolean sameAs(INodeDomain domain) {

+		return true;

+	}

 

-    @Override

-    public Integer cardinality() {

-        return 0;

-    }

+	@Override

+	public Integer cardinality() {

+		return 0;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
index fe4b921..cb4918d 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
@@ -32,74 +32,88 @@
 @SuppressWarnings("rawtypes")

 public class HiveMetaDataProvider<S, T> implements IMetadataProvider<S, T> {

 

-    private Operator fileSink;

-    private Schema outputSchema;

-    private HashMap<S, IDataSource<S>> dataSourceMap;

+	private Operator fileSink;

+	private Schema outputSchema;

+	private HashMap<S, IDataSource<S>> dataSourceMap;

 

-    public HiveMetaDataProvider(Operator fsOp, Schema oi, HashMap<S, IDataSource<S>> map) {

-        fileSink = fsOp;

-        outputSchema = oi;

-        dataSourceMap = map;

-    }

+	public HiveMetaDataProvider(Operator fsOp, Schema oi,

+			HashMap<S, IDataSource<S>> map) {

+		fileSink = fsOp;

+		outputSchema = oi;

+		dataSourceMap = map;

+	}

 

-    @Override

-    public IDataSourceIndex<T, S> findDataSourceIndex(T indexId, S dataSourceId) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public IDataSourceIndex<T, S> findDataSourceIndex(T indexId, S dataSourceId)

+			throws AlgebricksException {

+		return null;

+	}

 

-    @Override

-    public IDataSource<S> findDataSource(S id) throws AlgebricksException {

-        return dataSourceMap.get(id);

-    }

+	@Override

+	public IDataSource<S> findDataSource(S id) throws AlgebricksException {

+		return dataSourceMap.get(id);

+	}

 

-    @Override

-    public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource) {

-        return true;

-    }

+	@Override

+	public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource) {

+		return true;

+	}

 

-    @Override

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,

-            List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,

-            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)

-            throws AlgebricksException {

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(

+			IDataSource<S> dataSource, List<LogicalVariable> scanVariables,

+			List<LogicalVariable> projectVariables, boolean projectPushed,

+			IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,

+			JobGenContext context, JobSpecification jobSpec)

+			throws AlgebricksException {

 

-        S desc = dataSource.getId();

-        HiveScanRuntimeGenerator generator = new HiveScanRuntimeGenerator((PartitionDesc) desc);

-        return generator.getRuntimeOperatorAndConstraint(dataSource, scanVariables, projectVariables, projectPushed,

-                context, jobSpec);

-    }

+		S desc = dataSource.getId();

+		HiveScanRuntimeGenerator generator = new HiveScanRuntimeGenerator(

+				(PartitionDesc) desc);

+		return generator.getRuntimeOperatorAndConstraint(dataSource,

+				scanVariables, projectVariables, projectPushed, context,

+				jobSpec);

+	}

 

-    @Override

-    public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,

-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {

+	@Override

+	public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(

+			IDataSink sink, int[] printColumns,

+			IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {

 

-        HiveWriteRuntimeGenerator generator = new HiveWriteRuntimeGenerator((FileSinkOperator) fileSink, outputSchema);

-        return generator.getWriterRuntime(inputDesc);

-    }

+		HiveWriteRuntimeGenerator generator = new HiveWriteRuntimeGenerator(

+				(FileSinkOperator) fileSink, outputSchema);

+		return generator.getWriterRuntime(inputDesc);

+	}

 

-    @Override

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

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

-            JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(

+			IDataSource<S> arg0, IOperatorSchema arg1,

+			List<LogicalVariable> arg2, LogicalVariable arg3,

+			RecordDescriptor arg4, JobGenContext arg5, JobSpecification arg6)

+			throws AlgebricksException {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

-    @Override

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

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

-            JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(

+			IDataSource<S> arg0, IOperatorSchema arg1,

+			List<LogicalVariable> arg2, LogicalVariable arg3,

+			RecordDescriptor arg4, JobGenContext arg5, JobSpecification arg6)

+			throws AlgebricksException {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

-    @Override

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

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

-            JobSpecification arg5) throws AlgebricksException {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(

+			IDataSource<S> arg0, IOperatorSchema arg1,

+			List<LogicalVariable> arg2, LogicalVariable arg3,

+			JobGenContext arg4, JobSpecification arg5)

+			throws AlgebricksException {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

     @Override

     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,

@@ -107,29 +121,35 @@
         return null;

     }

 

-    @Override

-    public IFunctionInfo lookupFunction(FunctionIdentifier arg0) {

-        return new HiveFunctionInfo(arg0, null);

-    }

+   	@Override

+	public IFunctionInfo lookupFunction(FunctionIdentifier arg0) {

+		return new HiveFunctionInfo(arg0, null);

+	}

 

-    @Override

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(

-            IDataSourceIndex<T, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,

-            IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,

-            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)

-            throws AlgebricksException {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(

+			IDataSourceIndex<T, S> dataSource,

+			IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,

+			IVariableTypeEnvironment typeEnv,

+			List<LogicalVariable> primaryKeys,

+			List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr,

+			RecordDescriptor recordDesc, JobGenContext context,

+			JobSpecification spec) throws AlgebricksException {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

-    @Override

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(

-            IDataSourceIndex<T, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,

-            IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,

-            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)

-            throws AlgebricksException {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(

+			IDataSourceIndex<T, S> dataSource,

+			IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,

+			IVariableTypeEnvironment typeEnv,

+			List<LogicalVariable> primaryKeys,

+			List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr,

+			RecordDescriptor recordDesc, JobGenContext context,

+			JobSpecification spec) throws AlgebricksException {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java
index cdb0e95..83382f0 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveOperatorSchema.java
@@ -11,74 +11,74 @@
 

 public class HiveOperatorSchema implements IOperatorSchema {

 

-    private final Map<LogicalVariable, Integer> varMap;

+	private final Map<LogicalVariable, Integer> varMap;

 

-    private final List<LogicalVariable> varList;

+	private final List<LogicalVariable> varList;

 

-    public HiveOperatorSchema() {

-        varMap = new HashMap<LogicalVariable, Integer>();

-        varList = new ArrayList<LogicalVariable>();

-    }

+	public HiveOperatorSchema() {

+		varMap = new HashMap<LogicalVariable, Integer>();

+		varList = new ArrayList<LogicalVariable>();

+	}

 

-    @Override

-    public void addAllVariables(IOperatorSchema source) {

-        for (LogicalVariable v : source) {

-            varMap.put(v, varList.size());

-            varList.add(v);

-        }

-    }

+	@Override

+	public void addAllVariables(IOperatorSchema source) {

+		for (LogicalVariable v : source) {

+			varMap.put(v, varList.size());

+			varList.add(v);

+		}

+	}

 

-    @Override

-    public void addAllNewVariables(IOperatorSchema source) {

-        for (LogicalVariable v : source) {

-            if (varMap.get(v) == null) {

-                varMap.put(v, varList.size());

-                varList.add(v);

-            }

-        }

-    }

+	@Override

+	public void addAllNewVariables(IOperatorSchema source) {

+		for (LogicalVariable v : source) {

+			if (varMap.get(v) == null) {

+				varMap.put(v, varList.size());

+				varList.add(v);

+			}

+		}

+	}

 

-    @Override

-    public int addVariable(LogicalVariable var) {

-        int idx = varList.size();

-        varMap.put(var, idx);

-        varList.add(var);

-        return idx;

-    }

+	@Override

+	public int addVariable(LogicalVariable var) {

+		int idx = varList.size();

+		varMap.put(var, idx);

+		varList.add(var);

+		return idx;

+	}

 

-    @Override

-    public void clear() {

-        varMap.clear();

-        varList.clear();

-    }

+	@Override

+	public void clear() {

+		varMap.clear();

+		varList.clear();

+	}

 

-    @Override

-    public int findVariable(LogicalVariable var) {

-        Integer i = varMap.get(var);

-        if (i == null) {

-            return -1;

-        }

-        return i;

-    }

+	@Override

+	public int findVariable(LogicalVariable var) {

+		Integer i = varMap.get(var);

+		if (i == null) {

+			return -1;

+		}

+		return i;

+	}

 

-    @Override

-    public int getSize() {

-        return varList.size();

-    }

+	@Override

+	public int getSize() {

+		return varList.size();

+	}

 

-    @Override

-    public LogicalVariable getVariable(int index) {

-        return varList.get(index);

-    }

+	@Override

+	public LogicalVariable getVariable(int index) {

+		return varList.get(index);

+	}

 

-    @Override

-    public Iterator<LogicalVariable> iterator() {

-        return varList.iterator();

-    }

+	@Override

+	public Iterator<LogicalVariable> iterator() {

+		return varList.iterator();

+	}

 

-    @Override

-    public String toString() {

-        return varMap.toString();

-    }

+	@Override

+	public String toString() {

+		return varMap.toString();

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
index 69fb66a..9c8aee4 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
@@ -26,81 +26,92 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;

 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;

 

+@SuppressWarnings({ "rawtypes", "deprecation" })

 public class HiveScanRuntimeGenerator {

 

-    private PartitionDesc fileDesc;

+	private PartitionDesc fileDesc;

 

-    private transient Path filePath;

+	private transient Path filePath;

 

-    private String filePathName;

+	private String filePathName;

 

-    private Properties properties;

+	private Properties properties;

 

-    public HiveScanRuntimeGenerator(PartitionDesc path) {

-        fileDesc = path;

-        properties = fileDesc.getProperties();

+	public HiveScanRuntimeGenerator(PartitionDesc path) {

+		fileDesc = path;

+		properties = fileDesc.getProperties();

 

-        String inputPath = (String) properties.getProperty("location");

+		String inputPath = (String) properties.getProperty("location");

 

-        if (inputPath.startsWith("file:")) {

-            // Windows

-            String[] strs = inputPath.split(":");

-            filePathName = strs[strs.length - 1];

-        } else {

-            // Linux

-            filePathName = inputPath;

-        }

+		if (inputPath.startsWith("file:")) {

+			// Windows

+			String[] strs = inputPath.split(":");

+			filePathName = strs[strs.length - 1];

+		} else {

+			// Linux

+			filePathName = inputPath;

+		}

 

-        filePath = new Path(filePathName);

-    }

+		filePath = new Path(filePathName);

+	}

 

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRuntimeOperatorAndConstraint(

-            IDataSource dataSource, List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables,

-            boolean projectPushed, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {

-        // get the correct delimiter from Hive metastore or other data

-        // structures

-        IOperatorSchema propagatedSchema = new HiveOperatorSchema();

+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRuntimeOperatorAndConstraint(

+			IDataSource dataSource, List<LogicalVariable> scanVariables,

+			List<LogicalVariable> projectVariables, boolean projectPushed,

+			JobGenContext context, JobSpecification jobSpec)

+			throws AlgebricksException {

+		// get the correct delimiter from Hive metastore or other data

+		// structures

+		IOperatorSchema propagatedSchema = new HiveOperatorSchema();

 

-        List<LogicalVariable> outputVariables = projectPushed ? projectVariables : scanVariables;

-        for (LogicalVariable var : outputVariables)

-            propagatedSchema.addVariable(var);

+		List<LogicalVariable> outputVariables = projectPushed ? projectVariables

+				: scanVariables;

+		for (LogicalVariable var : outputVariables)

+			propagatedSchema.addVariable(var);

 

-        int[] outputColumnsOffset = new int[scanVariables.size()];

-        int i = 0;

-        for (LogicalVariable var : scanVariables)

-            if (outputVariables.contains(var)) {

-                int offset = outputVariables.indexOf(var);

-                outputColumnsOffset[i++] = offset;

-            } else

-                outputColumnsOffset[i++] = -1;

+		int[] outputColumnsOffset = new int[scanVariables.size()];

+		int i = 0;

+		for (LogicalVariable var : scanVariables)

+			if (outputVariables.contains(var)) {

+				int offset = outputVariables.indexOf(var);

+				outputColumnsOffset[i++] = offset;

+			} else

+				outputColumnsOffset[i++] = -1;

 

-        Object[] schemaTypes = dataSource.getSchemaTypes();

-        // get record descriptor

-        RecordDescriptor recDescriptor = mkRecordDescriptor(propagatedSchema, schemaTypes, context);

+		Object[] schemaTypes = dataSource.getSchemaTypes();

+		// get record descriptor

+		RecordDescriptor recDescriptor = mkRecordDescriptor(propagatedSchema,

+				schemaTypes, context);

 

-        // setup the run time operator

-        JobConf conf = ConfUtil.getJobConf(fileDesc.getInputFileFormatClass(), filePath);

-        int clusterSize = ConfUtil.getNCs().length;

-        IFileSplitProvider fsprovider = new HiveFileSplitProvider(conf, filePathName, clusterSize);

-        ITupleParserFactory tupleParserFactory = new HiveTupleParserFactory(fileDesc, conf, outputColumnsOffset);

-        HiveFileScanOperatorDescriptor opDesc = new HiveFileScanOperatorDescriptor(jobSpec, fsprovider,

-                tupleParserFactory, recDescriptor);

+		// setup the run time operator

+		JobConf conf = ConfUtil.getJobConf(fileDesc.getInputFileFormatClass(),

+				filePath);

+		int clusterSize = ConfUtil.getNCs().length;

+		IFileSplitProvider fsprovider = new HiveFileSplitProvider(conf,

+				filePathName, clusterSize);

+		ITupleParserFactory tupleParserFactory = new HiveTupleParserFactory(

+				fileDesc, conf, outputColumnsOffset);

+		HiveFileScanOperatorDescriptor opDesc = new HiveFileScanOperatorDescriptor(

+				jobSpec, fsprovider, tupleParserFactory, recDescriptor);

 

-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(opDesc, opDesc.getPartitionConstraint());

-    }

+		return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(

+				opDesc, opDesc.getPartitionConstraint());

+	}

 

-    @SuppressWarnings("unchecked")

-    private static RecordDescriptor mkRecordDescriptor(IOperatorSchema opSchema, Object[] types, JobGenContext context)

-            throws AlgebricksException {

-        ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema.getSize()];

-        ISerializerDeserializerProvider sdp = context.getSerializerDeserializerProvider();

-        int i = 0;

-        for (LogicalVariable var : opSchema) {

-            Object t = types[i];

-            fields[i] = sdp.getSerializerDeserializer(t);

-            i++;

-        }

-        return new RecordDescriptor(fields);

-    }

+	private static RecordDescriptor mkRecordDescriptor(

+			IOperatorSchema opSchema, Object[] types, JobGenContext context)

+			throws AlgebricksException {

+		ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema

+				.getSize()];

+		ISerializerDeserializerProvider sdp = context

+				.getSerializerDeserializerProvider();

+		int size = opSchema.getSize();

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

+			Object t = types[i];

+			fields[i] = sdp.getSerializerDeserializer(t);

+			i++;

+		}

+		return new RecordDescriptor(fields);

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java
index bd5bc7f..d372868 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveWriteRuntimeGenerator.java
@@ -11,27 +11,30 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;

 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;

 

+@SuppressWarnings("deprecation")

 public class HiveWriteRuntimeGenerator {

-    private FileSinkOperator fileSink;

+	private FileSinkOperator fileSink;

 

-    private Schema inputSchema;

+	private Schema inputSchema;

 

-    public HiveWriteRuntimeGenerator(FileSinkOperator fsOp, Schema oi) {

-        fileSink = fsOp;

-        inputSchema = oi;

-    }

+	public HiveWriteRuntimeGenerator(FileSinkOperator fsOp, Schema oi) {

+		fileSink = fsOp;

+		inputSchema = oi;

+	}

 

-    /**

-     * get the write runtime

-     * 

-     * @param inputDesc

-     * @return

-     */

-    public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriterRuntime(RecordDescriptor inputDesc) {

-        JobConf conf = ConfUtil.getJobConf();

-        IPushRuntimeFactory factory = new HivePushRuntimeFactory(inputDesc, conf, fileSink, inputSchema);

-        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> pair = new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(

-                factory, null);

-        return pair;

-    }

+	/**

+	 * get the write runtime

+	 * 

+	 * @param inputDesc

+	 * @return

+	 */

+	public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriterRuntime(

+			RecordDescriptor inputDesc) {

+		JobConf conf = ConfUtil.getJobConf();

+		IPushRuntimeFactory factory = new HivePushRuntimeFactory(inputDesc,

+				conf, fileSink, inputSchema);

+		Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> pair = new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(

+				factory, null);

+		return pair;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java
index 642280d..2f988f8 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveFileSplitProvider.java
@@ -3,14 +3,17 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;

 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;

 

-public abstract class AbstractHiveFileSplitProvider implements IFileSplitProvider {

+public abstract class AbstractHiveFileSplitProvider implements

+		IFileSplitProvider {

+	private static final long serialVersionUID = 1L;

 

-    @Override

-    public FileSplit[] getFileSplits() {

-        // TODO Auto-generated method stub

-        return null;

-    }

+	@Override

+	public FileSplit[] getFileSplits() {

+		// TODO Auto-generated method stub

+		return null;

+	}

 

-    public abstract org.apache.hadoop.mapred.FileSplit[] getFileSplitArray();

+	@SuppressWarnings("deprecation")

+	public abstract org.apache.hadoop.mapred.FileSplit[] getFileSplitArray();

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java
index a76a383..a8addeb 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/AbstractHiveTupleParser.java
@@ -8,19 +8,22 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;

 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;

 

+@SuppressWarnings("deprecation")

 public abstract class AbstractHiveTupleParser implements ITupleParser {

 

-    @Override

-    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {

-        // empty implementation

-    }

+	@Override

+	public void parse(InputStream in, IFrameWriter writer)

+			throws HyracksDataException {

+		// empty implementation

+	}

 

-    /**

-     * method for parsing HDFS file split

-     * 

-     * @param split

-     * @param writer

-     */

-    abstract public void parse(FileSplit split, IFrameWriter writer) throws HyracksDataException;

+	/**

+	 * method for parsing HDFS file split

+	 * 

+	 * @param split

+	 * @param writer

+	 */

+	abstract public void parse(FileSplit split, IFrameWriter writer)

+			throws HyracksDataException;

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java
index 00a9d0a..d248486 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileScanOperatorDescriptor.java
@@ -39,126 +39,135 @@
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
-public class HiveFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
+@SuppressWarnings("deprecation")
+public class HiveFileScanOperatorDescriptor extends
+		AbstractSingleActivityOperatorDescriptor {
+	private static final long serialVersionUID = 1L;
 
-    /**
-     * tuple parser factory
-     */
-    private final ITupleParserFactory tupleParserFactory;
+	/**
+	 * tuple parser factory
+	 */
+	private final ITupleParserFactory tupleParserFactory;
 
-    /**
-     * Hive file split
-     */
-    private Partition[] parts;
+	/**
+	 * Hive file split
+	 */
+	private Partition[] parts;
 
-    /**
-     * IFileSplitProvider
-     */
-    private IFileSplitProvider fileSplitProvider;
+	/**
+	 * IFileSplitProvider
+	 */
+	private IFileSplitProvider fileSplitProvider;
 
-    /**
-     * constrains in the form of host DNS names
-     */
-    private String[] constraintsByHostNames;
+	/**
+	 * constrains in the form of host DNS names
+	 */
+	private String[] constraintsByHostNames;
 
-    /**
-     * ip-to-node controller mapping
-     */
-    private Map<String, List<String>> ncMapping;
+	/**
+	 * ip-to-node controller mapping
+	 */
+	private Map<String, List<String>> ncMapping;
 
-    /**
-     * an array of NCs
-     */
-    private String[] NCs;
+	/**
+	 * an array of NCs
+	 */
+	private String[] NCs;
 
-    /**
-     * 
-     * @param spec
-     * @param fsProvider
-     */
-    public HiveFileScanOperatorDescriptor(JobSpecification spec, IFileSplitProvider fsProvider,
-            ITupleParserFactory tupleParserFactory, RecordDescriptor rDesc) {
-        super(spec, 0, 1);
-        this.tupleParserFactory = tupleParserFactory;
-        recordDescriptors[0] = rDesc;
-        fileSplitProvider = fsProvider;
-    }
+	/**
+	 * 
+	 * @param spec
+	 * @param fsProvider
+	 */
+	public HiveFileScanOperatorDescriptor(JobSpecification spec,
+			IFileSplitProvider fsProvider,
+			ITupleParserFactory tupleParserFactory, RecordDescriptor rDesc) {
+		super(spec, 0, 1);
+		this.tupleParserFactory = tupleParserFactory;
+		recordDescriptors[0] = rDesc;
+		fileSplitProvider = fsProvider;
+	}
 
-    /**
-     * set partition constraint at the first time it is called the number of
-     * partitions is obtained from HDFS name node
-     */
-    public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
-        FileSplit[] returnedSplits = ((AbstractHiveFileSplitProvider) fileSplitProvider).getFileSplitArray();
-        Random random = new Random(System.currentTimeMillis());
-        ncMapping = ConfUtil.getNCMapping();
-        NCs = ConfUtil.getNCs();
+	/**
+	 * set partition constraint at the first time it is called the number of
+	 * partitions is obtained from HDFS name node
+	 */
+	public AlgebricksAbsolutePartitionConstraint getPartitionConstraint()
+			throws AlgebricksException {
+		FileSplit[] returnedSplits = ((AbstractHiveFileSplitProvider) fileSplitProvider)
+				.getFileSplitArray();
+		Random random = new Random(System.currentTimeMillis());
+		ncMapping = ConfUtil.getNCMapping();
+		NCs = ConfUtil.getNCs();
 
-        int size = 0;
-        for (FileSplit split : returnedSplits)
-            if (split != null)
-                size++;
+		int size = 0;
+		for (FileSplit split : returnedSplits)
+			if (split != null)
+				size++;
 
-        FileSplit[] splits = new FileSplit[size];
-        for (int i = 0; i < returnedSplits.length; i++)
-            if (returnedSplits[i] != null)
-                splits[i] = returnedSplits[i];
+		FileSplit[] splits = new FileSplit[size];
+		for (int i = 0; i < returnedSplits.length; i++)
+			if (returnedSplits[i] != null)
+				splits[i] = returnedSplits[i];
 
-        System.out.println("!!! number of splits: " + splits.length);
-        constraintsByHostNames = new String[splits.length];
-        for (int i = 0; i < splits.length; i++) {
-            try {
-                String[] loc = splits[i].getLocations();
-                Collections.shuffle(Arrays.asList(loc), random);
-                if (loc.length > 0) {
-                    InetAddress[] allIps = InetAddress.getAllByName(loc[0]);
-                    for (InetAddress ip : allIps) {
-                        if (ncMapping.get(ip.getHostAddress()) != null) {
-                            List<String> ncs = ncMapping.get(ip.getHostAddress());
-                            int pos = random.nextInt(ncs.size());
-                            constraintsByHostNames[i] = ncs.get(pos);
-                        } else {
-                            int pos = random.nextInt(NCs.length);
-                            constraintsByHostNames[i] = NCs[pos];
-                        }
-                    }
-                } else {
-                    int pos = random.nextInt(NCs.length);
-                    constraintsByHostNames[i] = NCs[pos];
-                    if (splits[i].getLength() > 0)
-                        throw new IllegalStateException("non local scanner non locations!!");
-                }
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
-            }
-        }
+		System.out.println("!!! number of splits: " + splits.length);
+		constraintsByHostNames = new String[splits.length];
+		for (int i = 0; i < splits.length; i++) {
+			try {
+				String[] loc = splits[i].getLocations();
+				Collections.shuffle(Arrays.asList(loc), random);
+				if (loc.length > 0) {
+					InetAddress[] allIps = InetAddress.getAllByName(loc[0]);
+					for (InetAddress ip : allIps) {
+						if (ncMapping.get(ip.getHostAddress()) != null) {
+							List<String> ncs = ncMapping.get(ip
+									.getHostAddress());
+							int pos = random.nextInt(ncs.size());
+							constraintsByHostNames[i] = ncs.get(pos);
+						} else {
+							int pos = random.nextInt(NCs.length);
+							constraintsByHostNames[i] = NCs[pos];
+						}
+					}
+				} else {
+					int pos = random.nextInt(NCs.length);
+					constraintsByHostNames[i] = NCs[pos];
+					if (splits[i].getLength() > 0)
+						throw new IllegalStateException(
+								"non local scanner non locations!!");
+				}
+			} catch (IOException e) {
+				throw new AlgebricksException(e);
+			}
+		}
 
-        parts = new Partition[splits.length];
-        for (int i = 0; i < splits.length; i++) {
-            parts[i] = new Partition(splits[i]);
-        }
-        return new AlgebricksAbsolutePartitionConstraint(constraintsByHostNames);
-    }
+		parts = new Partition[splits.length];
+		for (int i = 0; i < splits.length; i++) {
+			parts[i] = new Partition(splits[i]);
+		}
+		return new AlgebricksAbsolutePartitionConstraint(constraintsByHostNames);
+	}
 
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+	@Override
+	public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+			IRecordDescriptorProvider recordDescProvider, int partition,
+			int nPartitions) {
 
-        final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
-        final int partitionId = partition;
+		final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
+		final int partitionId = partition;
 
-        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+		return new AbstractUnaryOutputSourceOperatorNodePushable() {
 
-            @Override
-            public void initialize() throws HyracksDataException {
-                writer.open();
-                FileSplit split = parts[partitionId].toFileSplit();
-                if (split == null)
-                    throw new HyracksDataException("partition " + partitionId + " is null!");
-                ((AbstractHiveTupleParser) tp).parse(split, writer);
-                writer.close();
-            }
-        };
-    }
+			@Override
+			public void initialize() throws HyracksDataException {
+				writer.open();
+				FileSplit split = parts[partitionId].toFileSplit();
+				if (split == null)
+					throw new HyracksDataException("partition " + partitionId
+							+ " is null!");
+				((AbstractHiveTupleParser) tp).parse(split, writer);
+				writer.close();
+			}
+		};
+	}
 }
\ No newline at end of file
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java
index 9e85ac4..d92d353 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveFileSplitProvider.java
@@ -15,94 +15,101 @@
 import org.apache.hadoop.mapred.JobConf;

 import org.eclipse.jetty.util.log.Log;

 

+@SuppressWarnings({ "deprecation", "rawtypes" })

 public class HiveFileSplitProvider extends AbstractHiveFileSplitProvider {

+	private static final long serialVersionUID = 1L;

 

-    private transient InputFormat format;

-    private transient JobConf conf;

-    private String confContent;

-    final private int nPartition;

-    private transient FileSplit[] splits;

-    private String inputFormatStr;

+	private transient InputFormat format;

+	private transient JobConf conf;

+	private String confContent;

+	final private int nPartition;

+	private transient FileSplit[] splits;

 

-    public HiveFileSplitProvider(JobConf conf, String filePath, int nPartition) {

-        format = conf.getInputFormat();

-        inputFormatStr = format.getClass().getName();

-        this.conf = conf;

-        this.nPartition = nPartition;

-        writeConfContent();

-    }

+	public HiveFileSplitProvider(JobConf conf, String filePath, int nPartition) {

+		format = conf.getInputFormat();

+		this.conf = conf;

+		this.nPartition = nPartition;

+		writeConfContent();

+	}

 

-    private void writeConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void writeConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));

-            conf.writeXml(out);

-            out.close();

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			DataOutputStream out = new DataOutputStream(new FileOutputStream(

+					new File(fileName)));

+			conf.writeXml(out);

+			out.close();

 

-            DataInputStream in = new DataInputStream(new FileInputStream(fileName));

-            StringBuffer buffer = new StringBuffer();

-            String line;

-            while ((line = in.readLine()) != null) {

-                buffer.append(line + "\n");

-            }

-            in.close();

-            confContent = buffer.toString();

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+			DataInputStream in = new DataInputStream(new FileInputStream(

+					fileName));

+			StringBuffer buffer = new StringBuffer();

+			String line;

+			while ((line = in.readLine()) != null) {

+				buffer.append(line + "\n");

+			}

+			in.close();

+			confContent = buffer.toString();

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    private void readConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void readConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));

-            out.write(confContent);

-            out.close();

-            conf = new JobConf(fileName);

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			PrintWriter out = new PrintWriter((new OutputStreamWriter(

+					new FileOutputStream(new File(fileName)))));

+			out.write(confContent);

+			out.close();

+			conf = new JobConf(fileName);

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    @Override

-    /**

-     * get the HDFS file split

-     */

-    public FileSplit[] getFileSplitArray() {

-        readConfContent();

-        conf.setClassLoader(this.getClass().getClassLoader());

-        format = conf.getInputFormat();

-        //int splitSize = conf.getInt("mapred.min.split.size", 0);

+	@Override

+	/**

+	 * get the HDFS file split

+	 */

+	public FileSplit[] getFileSplitArray() {

+		readConfContent();

+		conf.setClassLoader(this.getClass().getClassLoader());

+		format = conf.getInputFormat();

+		// int splitSize = conf.getInt("mapred.min.split.size", 0);

 

-        if (splits == null) {

-            try {

-                splits = (org.apache.hadoop.mapred.FileSplit[]) format.getSplits(conf, nPartition);

-                System.out.println("hdfs split number: " + splits.length);

-            } catch (IOException e) {

-                String inputPath = conf.get("mapred.input.dir");

-                String hdfsURL = conf.get("fs.default.name");

-                String alternatePath = inputPath.replaceAll(hdfsURL, "file:");

-                conf.set("mapred.input.dir", alternatePath);

-                try {

-                    splits = (org.apache.hadoop.mapred.FileSplit[]) format.getSplits(conf, nPartition);

-                    System.out.println("hdfs split number: " + splits.length);

-                } catch (IOException e1) {

-                    e1.printStackTrace();

-                    Log.debug(e1.getMessage());

-                    return null;

-                }

-            }

-        }

-        return splits;

-    }

+		if (splits == null) {

+			try {

+				splits = (org.apache.hadoop.mapred.FileSplit[]) format

+						.getSplits(conf, nPartition);

+				System.out.println("hdfs split number: " + splits.length);

+			} catch (IOException e) {

+				String inputPath = conf.get("mapred.input.dir");

+				String hdfsURL = conf.get("fs.default.name");

+				String alternatePath = inputPath.replaceAll(hdfsURL, "file:");

+				conf.set("mapred.input.dir", alternatePath);

+				try {

+					splits = (org.apache.hadoop.mapred.FileSplit[]) format

+							.getSplits(conf, nPartition);

+					System.out.println("hdfs split number: " + splits.length);

+				} catch (IOException e1) {

+					e1.printStackTrace();

+					Log.debug(e1.getMessage());

+					return null;

+				}

+			}

+		}

+		return splits;

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java
index ffcaa1d..7681bd1 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParser.java
@@ -33,187 +33,201 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;

 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;

 

+@SuppressWarnings({ "rawtypes", "deprecation", "unchecked" })

 public class HiveTupleParser extends AbstractHiveTupleParser {

 

-    private int[] outputColumnsOffset;

-    /**

-     * class of input format

-     */

-    private InputFormat inputFormat;

+	private int[] outputColumnsOffset;

+	/**

+	 * class of input format

+	 */

+	private InputFormat inputFormat;

 

-    /**

-     * serialization/deserialization object

-     */

-    private SerDe serDe;

+	/**

+	 * serialization/deserialization object

+	 */

+	private SerDe serDe;

 

-    /**

-     * the input row object inspector

-     */

-    private ObjectInspector objectInspector;

+	/**

+	 * the input row object inspector

+	 */

+	private ObjectInspector objectInspector;

 

-    /**

-     * the hadoop job conf

-     */

-    private JobConf job;

+	/**

+	 * the hadoop job conf

+	 */

+	private JobConf job;

 

-    /**

-     * Hyrax context to control resource allocation

-     */

-    private final IHyracksTaskContext ctx;

+	/**

+	 * Hyrax context to control resource allocation

+	 */

+	private final IHyracksTaskContext ctx;

 

-    /**

-     * lazy serde: format flow in between operators

-     */

-    private final SerDe outputSerDe;

+	/**

+	 * lazy serde: format flow in between operators

+	 */

+	private final SerDe outputSerDe;

 

-    /**

-     * the parser from hive data to binary data

-     */

-    private IHiveParser parser = null;

+	/**

+	 * the parser from hive data to binary data

+	 */

+	private IHiveParser parser = null;

 

-    /**

-     * parser for any hive input format

-     * 

-     * @param inputFormatClass

-     * @param serDeClass

-     * @param tbl

-     * @param conf

-     * @throws AlgebricksException

-     */

-    public HiveTupleParser(String inputFormatClass, String serDeClass, String outputSerDeClass, Properties tbl,

-            JobConf conf, final IHyracksTaskContext ctx, int[] outputColumnsOffset) throws AlgebricksException {

-        try {

-            conf.setClassLoader(this.getClass().getClassLoader());

+	/**

+	 * parser for any hive input format

+	 * 

+	 * @param inputFormatClass

+	 * @param serDeClass

+	 * @param tbl

+	 * @param conf

+	 * @throws AlgebricksException

+	 */

+	public HiveTupleParser(String inputFormatClass, String serDeClass,

+			String outputSerDeClass, Properties tbl, JobConf conf,

+			final IHyracksTaskContext ctx, int[] outputColumnsOffset)

+			throws AlgebricksException {

+		try {

+			conf.setClassLoader(this.getClass().getClassLoader());

 

-            inputFormat = (InputFormat) ReflectionUtils.newInstance(Class.forName(inputFormatClass), conf);

-            job = conf;

+			inputFormat = (InputFormat) ReflectionUtils.newInstance(

+					Class.forName(inputFormatClass), conf);

+			job = conf;

 

-            // initialize the input serde

-            serDe = (SerDe) ReflectionUtils.newInstance(Class.forName(serDeClass), job);

-            serDe.initialize(job, tbl);

+			// initialize the input serde

+			serDe = (SerDe) ReflectionUtils.newInstance(

+					Class.forName(serDeClass), job);

+			serDe.initialize(job, tbl);

 

-            // initialize the output serde

-            outputSerDe = (SerDe) ReflectionUtils.newInstance(Class.forName(outputSerDeClass), job);

-            outputSerDe.initialize(job, tbl);

+			// initialize the output serde

+			outputSerDe = (SerDe) ReflectionUtils.newInstance(

+					Class.forName(outputSerDeClass), job);

+			outputSerDe.initialize(job, tbl);

 

-            // object inspector of the row

-            objectInspector = serDe.getObjectInspector();

+			// object inspector of the row

+			objectInspector = serDe.getObjectInspector();

 

-            // hyracks context

-            this.ctx = ctx;

-            this.outputColumnsOffset = outputColumnsOffset;

+			// hyracks context

+			this.ctx = ctx;

+			this.outputColumnsOffset = outputColumnsOffset;

 

-            if (objectInspector instanceof LazySimpleStructObjectInspector) {

-                LazySimpleStructObjectInspector rowInspector = (LazySimpleStructObjectInspector) objectInspector;

-                List<? extends StructField> fieldRefs = rowInspector.getAllStructFieldRefs();

-                boolean lightWeightParsable = true;

-                for (StructField fieldRef : fieldRefs) {

-                    Category category = fieldRef.getFieldObjectInspector().getCategory();

-                    if (!(category == Category.PRIMITIVE)) {

-                        lightWeightParsable = false;

-                        break;

-                    }

-                }

-                if (lightWeightParsable)

-                    parser = new TextToBinaryTupleParser(this.outputColumnsOffset, this.objectInspector);

-            }

-        } catch (Exception e) {

-            throw new AlgebricksException(e);

-        }

-    }

+			if (objectInspector instanceof LazySimpleStructObjectInspector) {

+				LazySimpleStructObjectInspector rowInspector = (LazySimpleStructObjectInspector) objectInspector;

+				List<? extends StructField> fieldRefs = rowInspector

+						.getAllStructFieldRefs();

+				boolean lightWeightParsable = true;

+				for (StructField fieldRef : fieldRefs) {

+					Category category = fieldRef.getFieldObjectInspector()

+							.getCategory();

+					if (!(category == Category.PRIMITIVE)) {

+						lightWeightParsable = false;

+						break;

+					}

+				}

+				if (lightWeightParsable)

+					parser = new TextToBinaryTupleParser(

+							this.outputColumnsOffset, this.objectInspector);

+			}

+		} catch (Exception e) {

+			throw new AlgebricksException(e);

+		}

+	}

 

-    /**

-     * parse a input HDFS file split, the result is send to the writer

-     * one-frame-a-time

-     * 

-     * @param split

-     *            the HDFS file split

-     * @param writer

-     *            the writer

-     * @throws HyracksDataException

-     *             if there is sth. wrong in the ser/de

-     */

-    @Override

-    public void parse(FileSplit split, IFrameWriter writer) throws HyracksDataException {

-        try {

-            StructObjectInspector structInspector = (StructObjectInspector) objectInspector;

+	/**

+	 * parse a input HDFS file split, the result is send to the writer

+	 * one-frame-a-time

+	 * 

+	 * @param split

+	 *            the HDFS file split

+	 * @param writer

+	 *            the writer

+	 * @throws HyracksDataException

+	 *             if there is sth. wrong in the ser/de

+	 */

+	@Override

+	public void parse(FileSplit split, IFrameWriter writer)

+			throws HyracksDataException {

+		try {

+			StructObjectInspector structInspector = (StructObjectInspector) objectInspector;

 

-            // create the reader, key, and value

-            RecordReader reader = inputFormat.getRecordReader(split, job, Reporter.NULL);

-            Object key = reader.createKey();

-            Object value = reader.createValue();

+			// create the reader, key, and value

+			RecordReader reader = inputFormat.getRecordReader(split, job,

+					Reporter.NULL);

+			Object key = reader.createKey();

+			Object value = reader.createValue();

 

-            // allocate a new frame

-            ByteBuffer frame = ctx.allocateFrame();

-            FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());

-            appender.reset(frame, true);

+			// allocate a new frame

+			ByteBuffer frame = ctx.allocateFrame();

+			FrameTupleAppender appender = new FrameTupleAppender(

+					ctx.getFrameSize());

+			appender.reset(frame, true);

 

-            List<? extends StructField> fieldRefs = structInspector.getAllStructFieldRefs();

-            int size = 0;

-            for (int i = 0; i < outputColumnsOffset.length; i++)

-                if (outputColumnsOffset[i] >= 0)

-                    size++;

+			List<? extends StructField> fieldRefs = structInspector

+					.getAllStructFieldRefs();

+			int size = 0;

+			for (int i = 0; i < outputColumnsOffset.length; i++)

+				if (outputColumnsOffset[i] >= 0)

+					size++;

 

-            ArrayTupleBuilder tb = new ArrayTupleBuilder(size);

-            DataOutput dos = tb.getDataOutput();

-            StructField[] outputFieldRefs = new StructField[size];

-            Object[] outputFields = new Object[size];

-            for (int i = 0; i < outputColumnsOffset.length; i++)

-                if (outputColumnsOffset[i] >= 0)

-                    outputFieldRefs[outputColumnsOffset[i]] = fieldRefs.get(i);

+			ArrayTupleBuilder tb = new ArrayTupleBuilder(size);

+			DataOutput dos = tb.getDataOutput();

+			StructField[] outputFieldRefs = new StructField[size];

+			Object[] outputFields = new Object[size];

+			for (int i = 0; i < outputColumnsOffset.length; i++)

+				if (outputColumnsOffset[i] >= 0)

+					outputFieldRefs[outputColumnsOffset[i]] = fieldRefs.get(i);

 

-            long serDeTime = 0;

-            while (reader.next(key, value)) {

-                long start = System.currentTimeMillis();

-                // reuse the tuple builder

-                tb.reset();

-                if (parser != null) {

-                    Text text = (Text) value;

-                    parser.parse(text.getBytes(), 0, text.getLength(), tb);

-                } else {

-                    Object row = serDe.deserialize((Writable) value);

-                    // write fields to the tuple builder one by one

-                    int i = 0;

-                    for (StructField fieldRef : fieldRefs) {

-                        if (outputColumnsOffset[i] >= 0)

-                            outputFields[outputColumnsOffset[i]] = structInspector.getStructFieldData(row, fieldRef);

-                        i++;

-                    }

+			while (reader.next(key, value)) {

+				// reuse the tuple builder

+				tb.reset();

+				if (parser != null) {

+					Text text = (Text) value;

+					parser.parse(text.getBytes(), 0, text.getLength(), tb);

+				} else {

+					Object row = serDe.deserialize((Writable) value);

+					// write fields to the tuple builder one by one

+					int i = 0;

+					for (StructField fieldRef : fieldRefs) {

+						if (outputColumnsOffset[i] >= 0)

+							outputFields[outputColumnsOffset[i]] = structInspector

+									.getStructFieldData(row, fieldRef);

+						i++;

+					}

 

-                    i = 0;

-                    for (Object field : outputFields) {

-                        BytesWritable fieldWritable = (BytesWritable) outputSerDe.serialize(field,

-                                outputFieldRefs[i].getFieldObjectInspector());

-                        dos.write(fieldWritable.getBytes(), 0, fieldWritable.getSize());

-                        tb.addFieldEndOffset();

-                        i++;

-                    }

-                }

-                long end = System.currentTimeMillis();

-                serDeTime += (end - start);

+					i = 0;

+					for (Object field : outputFields) {

+						BytesWritable fieldWritable = (BytesWritable) outputSerDe

+								.serialize(field, outputFieldRefs[i]

+										.getFieldObjectInspector());

+						dos.write(fieldWritable.getBytes(), 0,

+								fieldWritable.getSize());

+						tb.addFieldEndOffset();

+						i++;

+					}

+				}

 

-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {

-                    if (appender.getTupleCount() <= 0)

-                        throw new IllegalStateException("zero tuples in a frame!");

-                    FrameUtils.flushFrame(frame, writer);

-                    appender.reset(frame, true);

-                    if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {

-                        throw new IllegalStateException();

-                    }

-                }

-            }

-            System.out.println("serde time: " + serDeTime);

-            reader.close();

-            System.gc();

+				if (!appender.append(tb.getFieldEndOffsets(),

+						tb.getByteArray(), 0, tb.getSize())) {

+					if (appender.getTupleCount() <= 0)

+						throw new IllegalStateException(

+								"zero tuples in a frame!");

+					FrameUtils.flushFrame(frame, writer);

+					appender.reset(frame, true);

+					if (!appender.append(tb.getFieldEndOffsets(),

+							tb.getByteArray(), 0, tb.getSize())) {

+						throw new IllegalStateException();

+					}

+				}

+			}

+			reader.close();

+			System.gc();

 

-            // flush the last frame

-            if (appender.getTupleCount() > 0) {

-                FrameUtils.flushFrame(frame, writer);

-            }

-        } catch (IOException e) {

-            throw new HyracksDataException(e);

-        } catch (SerDeException e) {

-            throw new HyracksDataException(e);

-        }

-    }

+			// flush the last frame

+			if (appender.getTupleCount() > 0) {

+				FrameUtils.flushFrame(frame, writer);

+			}

+		} catch (IOException e) {

+			throw new HyracksDataException(e);

+		} catch (SerDeException e) {

+			throw new HyracksDataException(e);

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java
index ccd18ee..69aa881 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/HiveTupleParserFactory.java
@@ -18,86 +18,94 @@
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;

 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;

 

+@SuppressWarnings("deprecation")

 public class HiveTupleParserFactory implements ITupleParserFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private int[] outputColumns;

+	private int[] outputColumns;

 

-    private String outputSerDeClass = LazySerDe.class.getName();

+	private String outputSerDeClass = LazySerDe.class.getName();

 

-    private String inputSerDeClass;

+	private String inputSerDeClass;

 

-    private transient JobConf conf;

+	private transient JobConf conf;

 

-    private Properties tbl;

+	private Properties tbl;

 

-    private String confContent;

+	private String confContent;

 

-    private String inputFormatClass;

+	private String inputFormatClass;

 

-    public HiveTupleParserFactory(PartitionDesc desc, JobConf conf, int[] outputColumns) {

-        this.conf = conf;

-        tbl = desc.getProperties();

-        inputFormatClass = (String) tbl.getProperty("file.inputformat");

-        inputSerDeClass = (String) tbl.getProperty("serialization.lib");

-        this.outputColumns = outputColumns;

+	public HiveTupleParserFactory(PartitionDesc desc, JobConf conf,

+			int[] outputColumns) {

+		this.conf = conf;

+		tbl = desc.getProperties();

+		inputFormatClass = (String) tbl.getProperty("file.inputformat");

+		inputSerDeClass = (String) tbl.getProperty("serialization.lib");

+		this.outputColumns = outputColumns;

 

-        writeConfContent();

-    }

+		writeConfContent();

+	}

 

-    @Override

-    public ITupleParser createTupleParser(IHyracksTaskContext ctx) {

-        readConfContent();

-        try {

-            return new HiveTupleParser(inputFormatClass, inputSerDeClass, outputSerDeClass, tbl, conf, ctx, outputColumns);

-        } catch (Exception e) {

-            e.printStackTrace();

-            return null;

-        }

-    }

+	@Override

+	public ITupleParser createTupleParser(IHyracksTaskContext ctx) {

+		readConfContent();

+		try {

+			return new HiveTupleParser(inputFormatClass, inputSerDeClass,

+					outputSerDeClass, tbl, conf, ctx, outputColumns);

+		} catch (Exception e) {

+			e.printStackTrace();

+			return null;

+		}

+	}

 

-    private void writeConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void writeConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));

-            conf.writeXml(out);

-            out.close();

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			DataOutputStream out = new DataOutputStream(new FileOutputStream(

+					new File(fileName)));

+			conf.writeXml(out);

+			out.close();

 

-            DataInputStream in = new DataInputStream(new FileInputStream(fileName));

-            StringBuffer buffer = new StringBuffer();

-            String line;

-            while ((line = in.readLine()) != null) {

-                buffer.append(line + "\n");

-            }

-            in.close();

-            confContent = buffer.toString();

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+			DataInputStream in = new DataInputStream(new FileInputStream(

+					fileName));

+			StringBuffer buffer = new StringBuffer();

+			String line;

+			while ((line = in.readLine()) != null) {

+				buffer.append(line + "\n");

+			}

+			in.close();

+			confContent = buffer.toString();

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    private void readConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void readConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));

-            out.write(confContent);

-            out.close();

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			PrintWriter out = new PrintWriter((new OutputStreamWriter(

+					new FileOutputStream(new File(fileName)))));

+			out.write(confContent);

+			out.close();

 

-            conf = new JobConf(fileName);

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+			conf = new JobConf(fileName);

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java
index db9dd78..1b3dcf2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filescan/Partition.java
@@ -6,28 +6,30 @@
 import org.apache.hadoop.fs.Path;

 import org.apache.hadoop.mapred.FileSplit;

 

+@SuppressWarnings("deprecation")

 public class Partition implements Serializable {

+	private static final long serialVersionUID = 1L;

 

-    private String uri;

-    private long offset;

-    private long length;

-    private String[] locations;

+	private String uri;

+	private long offset;

+	private long length;

+	private String[] locations;

 

-    public Partition() {

-    }

+	public Partition() {

+	}

 

-    public Partition(FileSplit file) {

-        uri = file.getPath().toUri().toString();

-        offset = file.getStart();

-        length = file.getLength();

-        try {

-            locations = file.getLocations();

-        } catch (IOException e) {

-            throw new IllegalStateException(e);

-        }

-    }

+	public Partition(FileSplit file) {

+		uri = file.getPath().toUri().toString();

+		offset = file.getStart();

+		length = file.getLength();

+		try {

+			locations = file.getLocations();

+		} catch (IOException e) {

+			throw new IllegalStateException(e);

+		}

+	}

 

-    public FileSplit toFileSplit() {

-        return new FileSplit(new Path(uri), offset, length, locations);

-    }

+	public FileSplit toFileSplit() {

+		return new FileSplit(new Path(uri), offset, length, locations);

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
index 1abade8..05e79ea 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
@@ -21,126 +21,133 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;

 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;

 

+@SuppressWarnings("deprecation")

 public class HiveFileWritePushRuntime implements IPushRuntime {

 

-    /**

-     * frame tuple accessor to access byte buffer

-     */

-    private final FrameTupleAccessor accessor;

+	/**

+	 * frame tuple accessor to access byte buffer

+	 */

+	private final FrameTupleAccessor accessor;

 

-    /**

-     * input object inspector

-     */

-    private final ObjectInspector inputInspector;

+	/**

+	 * input object inspector

+	 */

+	private final ObjectInspector inputInspector;

 

-    /**

-     * cachedInput

-     */

-    private final LazyColumnar cachedInput;

+	/**

+	 * cachedInput

+	 */

+	private final LazyColumnar cachedInput;

 

-    /**

-     * File sink operator of Hive

-     */

-    private final FileSinkDesc fileSink;

+	/**

+	 * File sink operator of Hive

+	 */

+	private final FileSinkDesc fileSink;

 

-    /**

-     * job configuration, which contain name node and other configuration

-     * information

-     */

-    private JobConf conf;

+	/**

+	 * job configuration, which contain name node and other configuration

+	 * information

+	 */

+	private JobConf conf;

 

-    /**

-     * input object inspector

-     */

-    private final Schema inputSchema;

+	/**

+	 * input object inspector

+	 */

+	private final Schema inputSchema;

 

-    /**

-     * a copy of hive schema representation

-     */

-    private RowSchema rowSchema;

+	/**

+	 * a copy of hive schema representation

+	 */

+	private RowSchema rowSchema;

 

-    /**

-     * the Hive file sink operator

-     */

-    private FileSinkOperator fsOp;

+	/**

+	 * the Hive file sink operator

+	 */

+	private FileSinkOperator fsOp;

 

-    /**

-     * cached tuple object reference

-     */

-    private FrameTupleReference tuple = new FrameTupleReference();

+	/**

+	 * cached tuple object reference

+	 */

+	private FrameTupleReference tuple = new FrameTupleReference();

 

-    /**

-     * @param spec

-     * @param fsProvider

-     */

-    public HiveFileWritePushRuntime(IHyracksTaskContext context, RecordDescriptor inputRecordDesc, JobConf job,

-            FileSinkDesc fs, RowSchema schema, Schema oi) {

-        fileSink = fs;

-        fileSink.setGatherStats(false);

+	/**

+	 * @param spec

+	 * @param fsProvider

+	 */

+	public HiveFileWritePushRuntime(IHyracksTaskContext context,

+			RecordDescriptor inputRecordDesc, JobConf job, FileSinkDesc fs,

+			RowSchema schema, Schema oi) {

+		fileSink = fs;

+		fileSink.setGatherStats(false);

 

-        rowSchema = schema;

-        conf = job;

-        inputSchema = oi;

+		rowSchema = schema;

+		conf = job;

+		inputSchema = oi;

 

-        accessor = new FrameTupleAccessor(context.getFrameSize(), inputRecordDesc);

-        inputInspector = inputSchema.toObjectInspector();

-        cachedInput = new LazyColumnar((LazyColumnarObjectInspector) inputInspector);

-    }

+		accessor = new FrameTupleAccessor(context.getFrameSize(),

+				inputRecordDesc);

+		inputInspector = inputSchema.toObjectInspector();

+		cachedInput = new LazyColumnar(

+				(LazyColumnarObjectInspector) inputInspector);

+	}

 

-    @Override

-    public void open() throws HyracksDataException {

-        fsOp = (FileSinkOperator) OperatorFactory.get(fileSink, rowSchema);

-        fsOp.setChildOperators(null);

-        fsOp.setParentOperators(null);

-        conf.setClassLoader(this.getClass().getClassLoader());

+	@Override

+	public void open() throws HyracksDataException {

+		fsOp = (FileSinkOperator) OperatorFactory.get(fileSink, rowSchema);

+		fsOp.setChildOperators(null);

+		fsOp.setParentOperators(null);

+		conf.setClassLoader(this.getClass().getClassLoader());

 

-        ObjectInspector[] inspectors = new ObjectInspector[1];

-        inspectors[0] = inputInspector;

-        try {

-            fsOp.initialize(conf, inspectors);

-            fsOp.setExecContext(null);

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+		ObjectInspector[] inspectors = new ObjectInspector[1];

+		inspectors[0] = inputInspector;

+		try {

+			fsOp.initialize(conf, inspectors);

+			fsOp.setExecContext(null);

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    @Override

-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {

-        accessor.reset(buffer);

-        int n = accessor.getTupleCount();

-        try {

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

-                tuple.reset(accessor, i);

-                cachedInput.init(tuple);

-                fsOp.process(cachedInput, 0);

-            }

-        } catch (HiveException e) {

-            throw new HyracksDataException(e);

-        }

-    }

+	@Override

+	public void nextFrame(ByteBuffer buffer) throws HyracksDataException {

+		accessor.reset(buffer);

+		int n = accessor.getTupleCount();

+		try {

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

+				tuple.reset(accessor, i);

+				cachedInput.init(tuple);

+				fsOp.process(cachedInput, 0);

+			}

+		} catch (HiveException e) {

+			throw new HyracksDataException(e);

+		}

+	}

 

-    @Override

-    public void close() throws HyracksDataException {

-        try {

-            Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());

-            fsOp.closeOp(false);

-        } catch (HiveException e) {

-            throw new HyracksDataException(e);

-        }

-    }

+	@Override

+	public void close() throws HyracksDataException {

+		try {

+			Thread.currentThread().setContextClassLoader(

+					this.getClass().getClassLoader());

+			fsOp.closeOp(false);

+		} catch (HiveException e) {

+			throw new HyracksDataException(e);

+		}

+	}

 

-    @Override

-    public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {

-        throw new IllegalStateException();

-    }

+	@Override

+	public void setFrameWriter(int index, IFrameWriter writer,

+			RecordDescriptor recordDesc) {

+		throw new IllegalStateException();

+	}

 

-    @Override

-    public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {

-    }

+	@Override

+	public void setInputRecordDescriptor(int index,

+			RecordDescriptor recordDescriptor) {

+	}

 

-    @Override

-    public void fail() throws HyracksDataException {

+	@Override

+	public void fail() throws HyracksDataException {

 

-    }

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java
index cdc5a36..43e90fa 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HivePushRuntimeFactory.java
@@ -21,84 +21,93 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;

 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;

 

+@SuppressWarnings("deprecation")

 public class HivePushRuntimeFactory implements IPushRuntimeFactory {

 

-    private static final long serialVersionUID = 1L;

+	private static final long serialVersionUID = 1L;

 

-    private final RecordDescriptor inputRecordDesc;

-    private transient JobConf conf;

-    private final FileSinkDesc fileSink;

-    private final RowSchema outSchema;

-    private final Schema schema;

+	private final RecordDescriptor inputRecordDesc;

+	private transient JobConf conf;

+	private final FileSinkDesc fileSink;

+	private final RowSchema outSchema;

+	private final Schema schema;

 

-    /**

-     * the content of the configuration

-     */

-    private String confContent;

+	/**

+	 * the content of the configuration

+	 */

+	private String confContent;

 

-    public HivePushRuntimeFactory(RecordDescriptor inputRecordDesc, JobConf conf, FileSinkOperator fsp, Schema sch) {

-        this.inputRecordDesc = inputRecordDesc;

-        this.conf = conf;

-        this.fileSink = fsp.getConf();

-        outSchema = fsp.getSchema();

-        this.schema = sch;

+	public HivePushRuntimeFactory(RecordDescriptor inputRecordDesc,

+			JobConf conf, FileSinkOperator fsp, Schema sch) {

+		this.inputRecordDesc = inputRecordDesc;

+		this.conf = conf;

+		this.fileSink = fsp.getConf();

+		outSchema = fsp.getSchema();

+		this.schema = sch;

 

-        writeConfContent();

-    }

+		writeConfContent();

+	}

 

-    @Override

-    public String toString() {

-        return "file write";

-    }

+	@Override

+	public String toString() {

+		return "file write";

+	}

 

-    @Override

-    public IPushRuntime createPushRuntime(IHyracksTaskContext context) throws AlgebricksException {

-        if (conf == null)

-            readConfContent();

+	@Override

+	public IPushRuntime createPushRuntime(IHyracksTaskContext context)

+			throws AlgebricksException {

+		if (conf == null)

+			readConfContent();

 

-        return new HiveFileWritePushRuntime(context, inputRecordDesc, conf, fileSink, outSchema, schema);

-    }

+		return new HiveFileWritePushRuntime(context, inputRecordDesc, conf,

+				fileSink, outSchema, schema);

+	}

 

-    private void readConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void readConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            PrintWriter out = new PrintWriter((new OutputStreamWriter(new FileOutputStream(new File(fileName)))));

-            out.write(confContent);

-            out.close();

-            conf = new JobConf(fileName);

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			PrintWriter out = new PrintWriter((new OutputStreamWriter(

+					new FileOutputStream(new File(fileName)))));

+			out.write(confContent);

+			out.close();

+			conf = new JobConf(fileName);

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    private void writeConfContent() {

-        File dir = new File("hadoop-conf-tmp");

-        if (!dir.exists()) {

-            dir.mkdir();

-        }

+	private void writeConfContent() {

+		File dir = new File("hadoop-conf-tmp");

+		if (!dir.exists()) {

+			dir.mkdir();

+		}

 

-        String fileName = "hadoop-conf-tmp/" + UUID.randomUUID() + System.currentTimeMillis() + ".xml";

-        try {

-            DataOutputStream out = new DataOutputStream(new FileOutputStream(new File(fileName)));

-            conf.writeXml(out);

-            out.close();

+		String fileName = "hadoop-conf-tmp/" + UUID.randomUUID()

+				+ System.currentTimeMillis() + ".xml";

+		try {

+			DataOutputStream out = new DataOutputStream(new FileOutputStream(

+					new File(fileName)));

+			conf.writeXml(out);

+			out.close();

 

-            DataInputStream in = new DataInputStream(new FileInputStream(fileName));

-            StringBuffer buffer = new StringBuffer();

-            String line;

-            while ((line = in.readLine()) != null) {

-                buffer.append(line + "\n");

-            }

-            in.close();

-            confContent = buffer.toString();

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+			DataInputStream in = new DataInputStream(new FileInputStream(

+					fileName));

+			StringBuffer buffer = new StringBuffer();

+			String line;

+			while ((line = in.readLine()) != null) {

+				buffer.append(line + "\n");

+			}

+			in.close();

+			confContent = buffer.toString();

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java
index 467ec0a..5a2e98c 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryComparatorFactoryProvider.java
@@ -21,55 +21,57 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;

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

 

-public class HiveBinaryComparatorFactoryProvider implements IBinaryComparatorFactoryProvider {

+public class HiveBinaryComparatorFactoryProvider implements

+		IBinaryComparatorFactoryProvider {

 

-    public static final HiveBinaryComparatorFactoryProvider INSTANCE = new HiveBinaryComparatorFactoryProvider();

+	public static final HiveBinaryComparatorFactoryProvider INSTANCE = new HiveBinaryComparatorFactoryProvider();

 

-    private HiveBinaryComparatorFactoryProvider() {

-    }

+	private HiveBinaryComparatorFactoryProvider() {

+	}

 

-    @Override

-    public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending)

-            throws AlgebricksException {

-        if (type.equals(TypeInfoFactory.intTypeInfo)) {

-            if (ascending)

-                return HiveIntegerBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveIntegerBinaryDescComparatorFactory.INSTANCE;

+	@Override

+	public IBinaryComparatorFactory getBinaryComparatorFactory(Object type,

+			boolean ascending) throws AlgebricksException {

+		if (type.equals(TypeInfoFactory.intTypeInfo)) {

+			if (ascending)

+				return HiveIntegerBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveIntegerBinaryDescComparatorFactory.INSTANCE;

 

-        } else if (type.equals(TypeInfoFactory.longTypeInfo)) {

-            if (ascending)

-                return HiveLongBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveLongBinaryDescComparatorFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.longTypeInfo)) {

+			if (ascending)

+				return HiveLongBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveLongBinaryDescComparatorFactory.INSTANCE;

 

-        } else if (type.equals(TypeInfoFactory.floatTypeInfo)) {

-            if (ascending)

-                return HiveFloatBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveFloatBinaryDescComparatorFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.floatTypeInfo)) {

+			if (ascending)

+				return HiveFloatBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveFloatBinaryDescComparatorFactory.INSTANCE;

 

-        } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {

-            if (ascending)

-                return HiveDoubleBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveDoubleBinaryDescComparatorFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.shortTypeInfo)) {

-            if (ascending)

-                return HiveShortBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveShortBinaryDescComparatorFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.stringTypeInfo)) {

-            if (ascending)

-                return HiveStringBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveStringBinaryDescComparatorFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.byteTypeInfo) || type.equals(TypeInfoFactory.booleanTypeInfo)) {

-            if (ascending)

-                return HiveByteBinaryAscComparatorFactory.INSTANCE;

-            else

-                return HiveByteBinaryDescComparatorFactory.INSTANCE;

-        } else

-            throw new NotImplementedException();

-    }

+		} else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {

+			if (ascending)

+				return HiveDoubleBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveDoubleBinaryDescComparatorFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.shortTypeInfo)) {

+			if (ascending)

+				return HiveShortBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveShortBinaryDescComparatorFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.stringTypeInfo)) {

+			if (ascending)

+				return HiveStringBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveStringBinaryDescComparatorFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.byteTypeInfo)

+				|| type.equals(TypeInfoFactory.booleanTypeInfo)) {

+			if (ascending)

+				return HiveByteBinaryAscComparatorFactory.INSTANCE;

+			else

+				return HiveByteBinaryDescComparatorFactory.INSTANCE;

+		} else

+			throw new NotImplementedException();

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java
index 473eee1..371d45b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFactoryProvider.java
@@ -11,25 +11,27 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;

 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;

 

-public class HiveBinaryHashFunctionFactoryProvider implements IBinaryHashFunctionFactoryProvider {

+public class HiveBinaryHashFunctionFactoryProvider implements

+		IBinaryHashFunctionFactoryProvider {

 

-    public static final HiveBinaryHashFunctionFactoryProvider INSTANCE = new HiveBinaryHashFunctionFactoryProvider();

+	public static final HiveBinaryHashFunctionFactoryProvider INSTANCE = new HiveBinaryHashFunctionFactoryProvider();

 

-    private HiveBinaryHashFunctionFactoryProvider() {

-    }

+	private HiveBinaryHashFunctionFactoryProvider() {

+	}

 

-    @Override

-    public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) throws AlgebricksException {

-        if (type.equals(TypeInfoFactory.intTypeInfo)) {

-            return HiveIntegerBinaryHashFunctionFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.longTypeInfo)) {

-            return HiveLongBinaryHashFunctionFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.stringTypeInfo)) {

-            return HiveStingBinaryHashFunctionFactory.INSTANCE;

-        } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {

-            return HiveDoubleBinaryHashFunctionFactory.INSTANCE;

-        } else {

-            return HiveRawBinaryHashFunctionFactory.INSTANCE;

-        }

-    }

+	@Override

+	public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type)

+			throws AlgebricksException {

+		if (type.equals(TypeInfoFactory.intTypeInfo)) {

+			return HiveIntegerBinaryHashFunctionFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.longTypeInfo)) {

+			return HiveLongBinaryHashFunctionFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.stringTypeInfo)) {

+			return HiveStingBinaryHashFunctionFactory.INSTANCE;

+		} else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {

+			return HiveDoubleBinaryHashFunctionFactory.INSTANCE;

+		} else {

+			return HiveRawBinaryHashFunctionFactory.INSTANCE;

+		}

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
new file mode 100644
index 0000000..e7a2e79
--- /dev/null
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hivesterix.runtime.provider;
+
+import edu.uci.ics.hivesterix.runtime.factory.hashfunction.MurmurHash3BinaryHashFunctionFamily;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+
+public class HiveBinaryHashFunctionFamilyProvider implements IBinaryHashFunctionFamilyProvider {
+
+    public static HiveBinaryHashFunctionFamilyProvider INSTANCE = new HiveBinaryHashFunctionFamilyProvider();
+
+    private HiveBinaryHashFunctionFamilyProvider() {
+
+    }
+
+    @Override
+    public IBinaryHashFunctionFamily getBinaryHashFunctionFamily(Object type) throws AlgebricksException {
+        return MurmurHash3BinaryHashFunctionFamily.INSTANCE;
+    }
+}
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java
index 91bf3e5..9e3a8ae 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveNormalizedKeyComputerFactoryProvider.java
@@ -13,39 +13,41 @@
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 
-public class HiveNormalizedKeyComputerFactoryProvider implements INormalizedKeyComputerFactoryProvider {
+public class HiveNormalizedKeyComputerFactoryProvider implements
+		INormalizedKeyComputerFactoryProvider {
 
-    public static final HiveNormalizedKeyComputerFactoryProvider INSTANCE = new HiveNormalizedKeyComputerFactoryProvider();
+	public static final HiveNormalizedKeyComputerFactoryProvider INSTANCE = new HiveNormalizedKeyComputerFactoryProvider();
 
-    private HiveNormalizedKeyComputerFactoryProvider() {
-    }
+	private HiveNormalizedKeyComputerFactoryProvider() {
+	}
 
-    @Override
-    public INormalizedKeyComputerFactory getNormalizedKeyComputerFactory(Object type, boolean ascending) {
-        if (ascending) {
-            if (type.equals(TypeInfoFactory.stringTypeInfo)) {
-                return new HiveStringAscNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.intTypeInfo)) {
-                return new HiveIntegerAscNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
-                return new HiveLongAscNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
-                return new HiveDoubleAscNormalizedKeyComputerFactory();
-            } else {
-                return null;
-            }
-        } else {
-            if (type.equals(TypeInfoFactory.stringTypeInfo)) {
-                return new HiveStringDescNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.intTypeInfo)) {
-                return new HiveIntegerDescNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.longTypeInfo)) {
-                return new HiveLongDescNormalizedKeyComputerFactory();
-            } else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
-                return new HiveDoubleDescNormalizedKeyComputerFactory();
-            } else {
-                return null;
-            }
-        }
-    }
+	@Override
+	public INormalizedKeyComputerFactory getNormalizedKeyComputerFactory(
+			Object type, boolean ascending) {
+		if (ascending) {
+			if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+				return new HiveStringAscNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.intTypeInfo)) {
+				return new HiveIntegerAscNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+				return new HiveLongAscNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+				return new HiveDoubleAscNormalizedKeyComputerFactory();
+			} else {
+				return null;
+			}
+		} else {
+			if (type.equals(TypeInfoFactory.stringTypeInfo)) {
+				return new HiveStringDescNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.intTypeInfo)) {
+				return new HiveIntegerDescNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.longTypeInfo)) {
+				return new HiveLongDescNormalizedKeyComputerFactory();
+			} else if (type.equals(TypeInfoFactory.doubleTypeInfo)) {
+				return new HiveDoubleDescNormalizedKeyComputerFactory();
+			} else {
+				return null;
+			}
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java
index 10c84d2..bebb457 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HivePrinterFactoryProvider.java
@@ -6,11 +6,12 @@
 

 public class HivePrinterFactoryProvider implements IPrinterFactoryProvider {

 

-    public static IPrinterFactoryProvider INSTANCE = new HivePrinterFactoryProvider();

+	public static IPrinterFactoryProvider INSTANCE = new HivePrinterFactoryProvider();

 

-    @Override

-    public IPrinterFactory getPrinterFactory(Object type) throws AlgebricksException {

-        return null;

-    }

+	@Override

+	public IPrinterFactory getPrinterFactory(Object type)

+			throws AlgebricksException {

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java
index c8b5699..7938de8 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveSerializerDeserializerProvider.java
@@ -4,18 +4,20 @@
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;

 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;

 

-public class HiveSerializerDeserializerProvider implements ISerializerDeserializerProvider {

+public class HiveSerializerDeserializerProvider implements

+		ISerializerDeserializerProvider {

 

-    public static final HiveSerializerDeserializerProvider INSTANCE = new HiveSerializerDeserializerProvider();

+	public static final HiveSerializerDeserializerProvider INSTANCE = new HiveSerializerDeserializerProvider();

 

-    private HiveSerializerDeserializerProvider() {

-    }

+	private HiveSerializerDeserializerProvider() {

+	}

 

-    @Override

-    public ISerializerDeserializer getSerializerDeserializer(Object type) throws AlgebricksException {

-        // TODO Auto-generated method stub

-        // return ARecordSerializerDeserializer.SCHEMALESS_INSTANCE;

-        return null;

-    }

+	@SuppressWarnings("rawtypes")

+	@Override

+	public ISerializerDeserializer getSerializerDeserializer(Object type)

+			throws AlgebricksException {

+		// return ARecordSerializerDeserializer.SCHEMALESS_INSTANCE;

+		return null;

+	}

 

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java
index 6ed1e63..2059128 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveTypeTraitProvider.java
@@ -6,28 +6,28 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class HiveTypeTraitProvider implements ITypeTraitProvider, Serializable {
+	private static final long serialVersionUID = 1L;
+	public static HiveTypeTraitProvider INSTANCE = new HiveTypeTraitProvider();
 
-    public static HiveTypeTraitProvider INSTANCE = new HiveTypeTraitProvider();
+	private HiveTypeTraitProvider() {
 
-    private HiveTypeTraitProvider() {
+	}
 
-    }
+	@Override
+	public ITypeTraits getTypeTrait(Object arg0) {
+		return new ITypeTraits() {
+			private static final long serialVersionUID = 1L;
 
-    @Override
-    public ITypeTraits getTypeTrait(Object arg0) {
-        return new ITypeTraits() {
-            private static final long serialVersionUID = 1L;
+			@Override
+			public int getFixedLength() {
+				return -1;
+			}
 
-            @Override
-            public int getFixedLength() {
-                return -1;
-            }
+			@Override
+			public boolean isFixedLength() {
+				return false;
+			}
 
-            @Override
-            public boolean isFixedLength() {
-                return false;
-            }
-
-        };
-    }
+		};
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java
index 9bf74b7..673416d 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/ByteArrayRef.java
@@ -26,17 +26,17 @@
  */
 public class ByteArrayRef {
 
-    /**
-     * Stores the actual data.
-     */
-    byte[] data;
+	/**
+	 * Stores the actual data.
+	 */
+	byte[] data;
 
-    public byte[] getData() {
-        return data;
-    }
+	public byte[] getData() {
+		return data;
+	}
 
-    public void setData(byte[] data) {
-        this.data = data;
-    }
+	public void setData(byte[] data) {
+		this.data = data;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java
index b79cd86..821c03d 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyArray.java
@@ -43,189 +43,194 @@
 
 public class LazyArray extends LazyNonPrimitive<LazyListObjectInspector> {
 
-    /**
-     * Whether the data is already parsed or not.
-     */
-    boolean parsed = false;
-    /**
-     * The length of the array. Only valid when the data is parsed.
-     */
-    int arraySize = 0;
+	/**
+	 * Whether the data is already parsed or not.
+	 */
+	boolean parsed = false;
+	/**
+	 * The length of the array. Only valid when the data is parsed.
+	 */
+	int arraySize = 0;
 
-    /**
-     * The start positions and lengths of array elements. Only valid when the
-     * data is parsed.
-     */
-    int[] elementStart;
-    int[] elementLength;
+	/**
+	 * The start positions and lengths of array elements. Only valid when the
+	 * data is parsed.
+	 */
+	int[] elementStart;
+	int[] elementLength;
 
-    /**
-     * Whether an element is initialized or not.
-     */
-    boolean[] elementInited;
+	/**
+	 * Whether an element is initialized or not.
+	 */
+	boolean[] elementInited;
 
-    /**
-     * Whether an element is null or not. Because length is 0 does not means the
-     * field is null. In particular, a 0-length string is not null.
-     */
-    boolean[] elementIsNull;
+	/**
+	 * Whether an element is null or not. Because length is 0 does not means the
+	 * field is null. In particular, a 0-length string is not null.
+	 */
+	boolean[] elementIsNull;
 
-    /**
-     * The elements of the array. Note that we call arrayElements[i].init(bytes,
-     * begin, length) only when that element is accessed.
-     */
-    LazyObject[] arrayElements;
+	/**
+	 * The elements of the array. Note that we call arrayElements[i].init(bytes,
+	 * begin, length) only when that element is accessed.
+	 */
+	@SuppressWarnings("rawtypes")
+	LazyObject[] arrayElements;
 
-    /**
-     * Construct a LazyArray object with the ObjectInspector.
-     * 
-     * @param oi
-     *            the oi representing the type of this LazyArray
-     */
-    protected LazyArray(LazyListObjectInspector oi) {
-        super(oi);
-    }
+	/**
+	 * Construct a LazyArray object with the ObjectInspector.
+	 * 
+	 * @param oi
+	 *            the oi representing the type of this LazyArray
+	 */
+	protected LazyArray(LazyListObjectInspector oi) {
+		super(oi);
+	}
 
-    /**
-     * Set the row data for this LazyArray.
-     * 
-     * @see LazyObject#init(ByteArrayRef, int, int)
-     */
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        super.init(bytes, start, length);
-        parsed = false;
-    }
+	/**
+	 * Set the row data for this LazyArray.
+	 * 
+	 * @see LazyObject#init(ByteArrayRef, int, int)
+	 */
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		super.init(bytes, start, length);
+		parsed = false;
+	}
 
-    /**
-     * Enlarge the size of arrays storing information for the elements inside
-     * the array.
-     */
-    private void adjustArraySize(int newSize) {
-        if (elementStart == null || elementStart.length < newSize) {
-            elementStart = new int[newSize];
-            elementLength = new int[newSize];
-            elementInited = new boolean[newSize];
-            elementIsNull = new boolean[newSize];
-            arrayElements = new LazyObject[newSize];
-        }
-    }
+	/**
+	 * Enlarge the size of arrays storing information for the elements inside
+	 * the array.
+	 */
+	private void adjustArraySize(int newSize) {
+		if (elementStart == null || elementStart.length < newSize) {
+			elementStart = new int[newSize];
+			elementLength = new int[newSize];
+			elementInited = new boolean[newSize];
+			elementIsNull = new boolean[newSize];
+			arrayElements = new LazyObject[newSize];
+		}
+	}
 
-    VInt vInt = new LazyUtils.VInt();
-    RecordInfo recordInfo = new LazyUtils.RecordInfo();
+	VInt vInt = new LazyUtils.VInt();
+	RecordInfo recordInfo = new LazyUtils.RecordInfo();
 
-    /**
-     * Parse the bytes and fill elementStart, elementLength, elementInited and
-     * elementIsNull.
-     */
-    private void parse() {
+	/**
+	 * Parse the bytes and fill elementStart, elementLength, elementInited and
+	 * elementIsNull.
+	 */
+	private void parse() {
 
-        // get the vlong that represents the map size
-        LazyUtils.readVInt(bytes, start, vInt);
-        arraySize = vInt.value;
-        if (0 == arraySize) {
-            parsed = true;
-            return;
-        }
+		// get the vlong that represents the map size
+		LazyUtils.readVInt(bytes, start, vInt);
+		arraySize = vInt.value;
+		if (0 == arraySize) {
+			parsed = true;
+			return;
+		}
 
-        // adjust arrays
-        adjustArraySize(arraySize);
-        // find out the null-bytes
-        int arryByteStart = start + vInt.length;
-        int nullByteCur = arryByteStart;
-        int nullByteEnd = arryByteStart + (arraySize + 7) / 8;
-        // the begin the real elements
-        int lastElementByteEnd = nullByteEnd;
-        // the list element object inspector
-        ObjectInspector listEleObjectInspector = ((ListObjectInspector) oi).getListElementObjectInspector();
-        // parsing elements one by one
-        for (int i = 0; i < arraySize; i++) {
-            elementIsNull[i] = true;
-            if ((bytes[nullByteCur] & (1 << (i % 8))) != 0) {
-                elementIsNull[i] = false;
-                LazyUtils.checkObjectByteInfo(listEleObjectInspector, bytes, lastElementByteEnd, recordInfo);
-                elementStart[i] = lastElementByteEnd + recordInfo.elementOffset;
-                elementLength[i] = recordInfo.elementSize;
-                lastElementByteEnd = elementStart[i] + elementLength[i];
-            }
-            // move onto the next null byte
-            if (7 == (i % 8)) {
-                nullByteCur++;
-            }
-        }
+		// adjust arrays
+		adjustArraySize(arraySize);
+		// find out the null-bytes
+		int arryByteStart = start + vInt.length;
+		int nullByteCur = arryByteStart;
+		int nullByteEnd = arryByteStart + (arraySize + 7) / 8;
+		// the begin the real elements
+		int lastElementByteEnd = nullByteEnd;
+		// the list element object inspector
+		ObjectInspector listEleObjectInspector = ((ListObjectInspector) oi)
+				.getListElementObjectInspector();
+		// parsing elements one by one
+		for (int i = 0; i < arraySize; i++) {
+			elementIsNull[i] = true;
+			if ((bytes[nullByteCur] & (1 << (i % 8))) != 0) {
+				elementIsNull[i] = false;
+				LazyUtils.checkObjectByteInfo(listEleObjectInspector, bytes,
+						lastElementByteEnd, recordInfo);
+				elementStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+				elementLength[i] = recordInfo.elementSize;
+				lastElementByteEnd = elementStart[i] + elementLength[i];
+			}
+			// move onto the next null byte
+			if (7 == (i % 8)) {
+				nullByteCur++;
+			}
+		}
 
-        Arrays.fill(elementInited, 0, arraySize, false);
-        parsed = true;
-    }
+		Arrays.fill(elementInited, 0, arraySize, false);
+		parsed = true;
+	}
 
-    /**
-     * Returns the actual primitive object at the index position inside the
-     * array represented by this LazyObject.
-     */
-    public Object getListElementObject(int index) {
-        if (!parsed) {
-            parse();
-        }
-        if (index < 0 || index >= arraySize) {
-            return null;
-        }
-        return uncheckedGetElement(index);
-    }
+	/**
+	 * Returns the actual primitive object at the index position inside the
+	 * array represented by this LazyObject.
+	 */
+	public Object getListElementObject(int index) {
+		if (!parsed) {
+			parse();
+		}
+		if (index < 0 || index >= arraySize) {
+			return null;
+		}
+		return uncheckedGetElement(index);
+	}
 
-    /**
-     * Get the element without checking out-of-bound index.
-     * 
-     * @param index
-     *            index to the array element
-     */
-    private Object uncheckedGetElement(int index) {
+	/**
+	 * Get the element without checking out-of-bound index.
+	 * 
+	 * @param index
+	 *            index to the array element
+	 */
+	private Object uncheckedGetElement(int index) {
 
-        if (elementIsNull[index]) {
-            return null;
-        } else {
-            if (!elementInited[index]) {
-                elementInited[index] = true;
-                if (arrayElements[index] == null) {
-                    arrayElements[index] = LazyFactory.createLazyObject((oi).getListElementObjectInspector());
-                }
-                arrayElements[index].init(bytes, elementStart[index], elementLength[index]);
-            }
-        }
-        return arrayElements[index].getObject();
-    }
+		if (elementIsNull[index]) {
+			return null;
+		} else {
+			if (!elementInited[index]) {
+				elementInited[index] = true;
+				if (arrayElements[index] == null) {
+					arrayElements[index] = LazyFactory.createLazyObject((oi)
+							.getListElementObjectInspector());
+				}
+				arrayElements[index].init(bytes, elementStart[index],
+						elementLength[index]);
+			}
+		}
+		return arrayElements[index].getObject();
+	}
 
-    /**
-     * Returns the array size.
-     */
-    public int getListLength() {
-        if (!parsed) {
-            parse();
-        }
-        return arraySize;
-    }
+	/**
+	 * Returns the array size.
+	 */
+	public int getListLength() {
+		if (!parsed) {
+			parse();
+		}
+		return arraySize;
+	}
 
-    /**
-     * cachedList is reused every time getList is called. Different
-     * LazyBianryArray instances cannot share the same cachedList.
-     */
-    ArrayList<Object> cachedList;
+	/**
+	 * cachedList is reused every time getList is called. Different
+	 * LazyBianryArray instances cannot share the same cachedList.
+	 */
+	ArrayList<Object> cachedList;
 
-    /**
-     * Returns the List of actual primitive objects. Returns null for null
-     * array.
-     */
-    public List<Object> getList() {
-        if (!parsed) {
-            parse();
-        }
-        if (cachedList == null) {
-            cachedList = new ArrayList<Object>(arraySize);
-        } else {
-            cachedList.clear();
-        }
-        for (int index = 0; index < arraySize; index++) {
-            cachedList.add(uncheckedGetElement(index));
-        }
-        return cachedList;
-    }
+	/**
+	 * Returns the List of actual primitive objects. Returns null for null
+	 * array.
+	 */
+	public List<Object> getList() {
+		if (!parsed) {
+			parse();
+		}
+		if (cachedList == null) {
+			cachedList = new ArrayList<Object>(arraySize);
+		} else {
+			cachedList.clear();
+		}
+		for (int index = 0; index < arraySize; index++) {
+			cachedList.add(uncheckedGetElement(index));
+		}
+		return cachedList;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java
index 471a64a..83b6254 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyBoolean.java
@@ -34,34 +34,35 @@
  * </p>
  * 
  */
-public class LazyBoolean extends LazyPrimitive<LazyBooleanObjectInspector, BooleanWritable> {
+public class LazyBoolean extends
+		LazyPrimitive<LazyBooleanObjectInspector, BooleanWritable> {
 
-    public LazyBoolean(LazyBooleanObjectInspector oi) {
-        super(oi);
-        data = new BooleanWritable();
-    }
+	public LazyBoolean(LazyBooleanObjectInspector oi) {
+		super(oi);
+		data = new BooleanWritable();
+	}
 
-    public LazyBoolean(LazyBoolean copy) {
-        super(copy);
-        data = new BooleanWritable(copy.data.get());
-    }
+	public LazyBoolean(LazyBoolean copy) {
+		super(copy);
+		data = new BooleanWritable(copy.data.get());
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
-        
-        // a temporal hack
-        assert (1 == length);
-        byte val = bytes[start];
-        if (val == 0) {
-            data.set(false);
-        } else if (val == 1) {
-            data.set(true);
-        }
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
+
+		// a temporal hack
+		assert (1 == length);
+		byte val = bytes[start];
+		if (val == 0) {
+			data.set(false);
+		} else if (val == 1) {
+			data.set(true);
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java
index d57dcb8..264015b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyByte.java
@@ -34,28 +34,29 @@
  * </p>
  * 
  */
-public class LazyByte extends LazyPrimitive<LazyByteObjectInspector, ByteWritable> {
+public class LazyByte extends
+		LazyPrimitive<LazyByteObjectInspector, ByteWritable> {
 
-    public LazyByte(LazyByteObjectInspector oi) {
-        super(oi);
-        data = new ByteWritable();
-    }
+	public LazyByte(LazyByteObjectInspector oi) {
+		super(oi);
+		data = new ByteWritable();
+	}
 
-    public LazyByte(LazyByte copy) {
-        super(copy);
-        data = new ByteWritable(copy.data.get());
-    }
+	public LazyByte(LazyByte copy) {
+		super(copy);
+		data = new ByteWritable(copy.data.get());
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
-        
-        assert (1 == length);
-        data.set(bytes[start]);
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
+
+		assert (1 == length);
+		data.set(bytes[start]);
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java
index 2ca9ded..a25ae49 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyColumnar.java
@@ -21,8 +21,6 @@
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
@@ -36,177 +34,178 @@
  * LazyStruct does not deal with the case of a NULL struct. That is handled by
  * the parent LazyObject.
  */
+@SuppressWarnings("rawtypes")
 public class LazyColumnar extends LazyNonPrimitive<LazyColumnarObjectInspector> {
 
-    private static Log LOG = LogFactory.getLog(LazyColumnar.class.getName());
+	/**
+	 * IFrameTupleReference: the backend of the struct
+	 */
+	IFrameTupleReference tuple;
 
-    /**
-     * IFrameTupleReference: the backend of the struct
-     */
-    IFrameTupleReference tuple;
+	/**
+	 * Whether the data is already parsed or not.
+	 */
+	boolean reset;
 
-    /**
-     * Whether the data is already parsed or not.
-     */
-    boolean reset;
+	/**
+	 * The fields of the struct.
+	 */
+	LazyObject[] fields;
 
-    /**
-     * The fields of the struct.
-     */
-    LazyObject[] fields;
+	/**
+	 * Whether init() has been called on the field or not.
+	 */
+	boolean[] fieldVisited;
 
-    /**
-     * Whether init() has been called on the field or not.
-     */
-    boolean[] fieldVisited;
+	/**
+	 * whether it is the first time initialization
+	 */
+	boolean start = true;
 
-    /**
-     * whether it is the first time initialization
-     */
-    boolean start = true;
+	/**
+	 * Construct a LazyStruct object with the ObjectInspector.
+	 */
+	public LazyColumnar(LazyColumnarObjectInspector oi) {
+		super(oi);
+	}
 
-    /**
-     * Construct a LazyStruct object with the ObjectInspector.
-     */
-    public LazyColumnar(LazyColumnarObjectInspector oi) {
-        super(oi);
-    }
+	/**
+	 * Set the row data for this LazyStruct.
+	 * 
+	 * @see LazyObject#init(ByteArrayRef, int, int)
+	 */
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		super.init(bytes, start, length);
+		reset = false;
+	}
 
-    /**
-     * Set the row data for this LazyStruct.
-     * 
-     * @see LazyObject#init(ByteArrayRef, int, int)
-     */
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        super.init(bytes, start, length);
-        reset = false;
-    }
+	/**
+	 * Parse the byte[] and fill each field.
+	 */
+	private void parse() {
 
-    /**
-     * Parse the byte[] and fill each field.
-     */
-    private void parse() {
+		if (start) {
+			// initialize field array and reusable objects
+			List<? extends StructField> fieldRefs = ((StructObjectInspector) oi)
+					.getAllStructFieldRefs();
 
-        if (start) {
-            // initialize field array and reusable objects
-            List<? extends StructField> fieldRefs = ((StructObjectInspector) oi).getAllStructFieldRefs();
+			fields = new LazyObject[fieldRefs.size()];
+			for (int i = 0; i < fields.length; i++) {
+				fields[i] = LazyFactory.createLazyObject(fieldRefs.get(i)
+						.getFieldObjectInspector());
+			}
+			fieldVisited = new boolean[fields.length];
+			start = false;
+		}
 
-            fields = new LazyObject[fieldRefs.size()];
-            for (int i = 0; i < fields.length; i++) {
-                fields[i] = LazyFactory.createLazyObject(fieldRefs.get(i).getFieldObjectInspector());
-            }
-            fieldVisited = new boolean[fields.length];
-            start = false;
-        }
+		Arrays.fill(fieldVisited, false);
+		reset = true;
+	}
 
-        Arrays.fill(fieldVisited, false);
-        reset = true;
-    }
+	/**
+	 * Get one field out of the struct.
+	 * 
+	 * If the field is a primitive field, return the actual object. Otherwise
+	 * return the LazyObject. This is because PrimitiveObjectInspector does not
+	 * have control over the object used by the user - the user simply directly
+	 * use the Object instead of going through Object
+	 * PrimitiveObjectInspector.get(Object).
+	 * 
+	 * @param fieldID
+	 *            The field ID
+	 * @return The field as a LazyObject
+	 */
+	public Object getField(int fieldID) {
+		if (!reset) {
+			parse();
+		}
+		return uncheckedGetField(fieldID);
+	}
 
-    /**
-     * Get one field out of the struct.
-     * 
-     * If the field is a primitive field, return the actual object. Otherwise
-     * return the LazyObject. This is because PrimitiveObjectInspector does not
-     * have control over the object used by the user - the user simply directly
-     * use the Object instead of going through Object
-     * PrimitiveObjectInspector.get(Object).
-     * 
-     * @param fieldID
-     *            The field ID
-     * @return The field as a LazyObject
-     */
-    public Object getField(int fieldID) {
-        if (!reset) {
-            parse();
-        }
-        return uncheckedGetField(fieldID);
-    }
+	/**
+	 * Get the field out of the row without checking parsed. This is called by
+	 * both getField and getFieldsAsList.
+	 * 
+	 * @param fieldID
+	 *            The id of the field starting from 0.
+	 * @param nullSequence
+	 *            The sequence representing NULL value.
+	 * @return The value of the field
+	 */
+	private Object uncheckedGetField(int fieldID) {
+		// get the buffer
+		byte[] buffer = tuple.getFieldData(fieldID);
+		// get the offset of the field
+		int s1 = tuple.getFieldStart(fieldID);
+		int l1 = tuple.getFieldLength(fieldID);
 
-    /**
-     * Get the field out of the row without checking parsed. This is called by
-     * both getField and getFieldsAsList.
-     * 
-     * @param fieldID
-     *            The id of the field starting from 0.
-     * @param nullSequence
-     *            The sequence representing NULL value.
-     * @return The value of the field
-     */
-    private Object uncheckedGetField(int fieldID) {
-        // get the buffer
-        byte[] buffer = tuple.getFieldData(fieldID);
-        // get the offset of the field
-        int s1 = tuple.getFieldStart(fieldID);
-        int l1 = tuple.getFieldLength(fieldID);
+		if (!fieldVisited[fieldID]) {
+			fieldVisited[fieldID] = true;
+			fields[fieldID].init(buffer, s1, l1);
+		}
+		// if (fields[fieldID].getObject() == null) {
+		// throw new IllegalStateException("illegal field " + fieldID);
+		// }
+		return fields[fieldID].getObject();
+	}
 
-        if (!fieldVisited[fieldID]) {
-            fieldVisited[fieldID] = true;
-            fields[fieldID].init(buffer, s1, l1);
-        }
-        //if (fields[fieldID].getObject() == null) {
-        //    throw new IllegalStateException("illegal field " + fieldID);
-        //}
-        return fields[fieldID].getObject();
-    }
+	ArrayList<Object> cachedList;
 
-    ArrayList<Object> cachedList;
+	/**
+	 * Get the values of the fields as an ArrayList.
+	 * 
+	 * @return The values of the fields as an ArrayList.
+	 */
+	public ArrayList<Object> getFieldsAsList() {
+		if (!reset) {
+			parse();
+		}
+		if (cachedList == null) {
+			cachedList = new ArrayList<Object>();
+		} else {
+			cachedList.clear();
+		}
+		for (int i = 0; i < fields.length; i++) {
+			cachedList.add(uncheckedGetField(i));
+		}
+		return cachedList;
+	}
 
-    /**
-     * Get the values of the fields as an ArrayList.
-     * 
-     * @return The values of the fields as an ArrayList.
-     */
-    public ArrayList<Object> getFieldsAsList() {
-        if (!reset) {
-            parse();
-        }
-        if (cachedList == null) {
-            cachedList = new ArrayList<Object>();
-        } else {
-            cachedList.clear();
-        }
-        for (int i = 0; i < fields.length; i++) {
-            cachedList.add(uncheckedGetField(i));
-        }
-        return cachedList;
-    }
+	@Override
+	public Object getObject() {
+		return this;
+	}
 
-    @Override
-    public Object getObject() {
-        return this;
-    }
+	protected boolean getParsed() {
+		return reset;
+	}
 
-    protected boolean getParsed() {
-        return reset;
-    }
+	protected void setParsed(boolean parsed) {
+		this.reset = parsed;
+	}
 
-    protected void setParsed(boolean parsed) {
-        this.reset = parsed;
-    }
+	protected LazyObject[] getFields() {
+		return fields;
+	}
 
-    protected LazyObject[] getFields() {
-        return fields;
-    }
+	protected void setFields(LazyObject[] fields) {
+		this.fields = fields;
+	}
 
-    protected void setFields(LazyObject[] fields) {
-        this.fields = fields;
-    }
+	protected boolean[] getFieldInited() {
+		return fieldVisited;
+	}
 
-    protected boolean[] getFieldInited() {
-        return fieldVisited;
-    }
+	protected void setFieldInited(boolean[] fieldInited) {
+		this.fieldVisited = fieldInited;
+	}
 
-    protected void setFieldInited(boolean[] fieldInited) {
-        this.fieldVisited = fieldInited;
-    }
-
-    /**
-     * rebind a frametuplereference to the struct
-     */
-    public void init(IFrameTupleReference r) {
-        this.tuple = r;
-        reset = false;
-    }
+	/**
+	 * rebind a frametuplereference to the struct
+	 */
+	public void init(IFrameTupleReference r) {
+		this.tuple = r;
+		reset = false;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java
index f7bb633..d687aa1 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyDouble.java
@@ -25,27 +25,29 @@
  * LazyObject for storing a value of Double.
  * 
  */
-public class LazyDouble extends LazyPrimitive<LazyDoubleObjectInspector, DoubleWritable> {
+public class LazyDouble extends
+		LazyPrimitive<LazyDoubleObjectInspector, DoubleWritable> {
 
-    public LazyDouble(LazyDoubleObjectInspector oi) {
-        super(oi);
-        data = new DoubleWritable();
-    }
+	public LazyDouble(LazyDoubleObjectInspector oi) {
+		super(oi);
+		data = new DoubleWritable();
+	}
 
-    public LazyDouble(LazyDouble copy) {
-        super(copy);
-        data = new DoubleWritable(copy.data.get());
-    }
+	public LazyDouble(LazyDouble copy) {
+		super(copy);
+		data = new DoubleWritable(copy.data.get());
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
-        assert (8 == length);
-        data.set(Double.longBitsToDouble(LazyUtils.byteArrayToLong(bytes, start)));
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
+		assert (8 == length);
+		data.set(Double.longBitsToDouble(LazyUtils
+				.byteArrayToLong(bytes, start)));
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java
index a249d61..e7593e4 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFactory.java
@@ -40,56 +40,58 @@
  */
 public final class LazyFactory {
 
-    /**
-     * Create a lazy binary primitive class given the type name.
-     */
-    public static LazyPrimitive<?, ?> createLazyPrimitiveClass(PrimitiveObjectInspector oi) {
-        PrimitiveCategory p = oi.getPrimitiveCategory();
-        switch (p) {
-            case BOOLEAN:
-                return new LazyBoolean((LazyBooleanObjectInspector) oi);
-            case BYTE:
-                return new LazyByte((LazyByteObjectInspector) oi);
-            case SHORT:
-                return new LazyShort((LazyShortObjectInspector) oi);
-            case INT:
-                return new LazyInteger((LazyIntObjectInspector) oi);
-            case LONG:
-                return new LazyLong((LazyLongObjectInspector) oi);
-            case FLOAT:
-                return new LazyFloat((LazyFloatObjectInspector) oi);
-            case DOUBLE:
-                return new LazyDouble((LazyDoubleObjectInspector) oi);
-            case STRING:
-                return new LazyString((LazyStringObjectInspector) oi);
-            default:
-                throw new RuntimeException("Internal error: no LazyObject for " + p);
-        }
-    }
+	/**
+	 * Create a lazy binary primitive class given the type name.
+	 */
+	public static LazyPrimitive<?, ?> createLazyPrimitiveClass(
+			PrimitiveObjectInspector oi) {
+		PrimitiveCategory p = oi.getPrimitiveCategory();
+		switch (p) {
+		case BOOLEAN:
+			return new LazyBoolean((LazyBooleanObjectInspector) oi);
+		case BYTE:
+			return new LazyByte((LazyByteObjectInspector) oi);
+		case SHORT:
+			return new LazyShort((LazyShortObjectInspector) oi);
+		case INT:
+			return new LazyInteger((LazyIntObjectInspector) oi);
+		case LONG:
+			return new LazyLong((LazyLongObjectInspector) oi);
+		case FLOAT:
+			return new LazyFloat((LazyFloatObjectInspector) oi);
+		case DOUBLE:
+			return new LazyDouble((LazyDoubleObjectInspector) oi);
+		case STRING:
+			return new LazyString((LazyStringObjectInspector) oi);
+		default:
+			throw new RuntimeException("Internal error: no LazyObject for " + p);
+		}
+	}
 
-    /**
-     * Create a hierarchical LazyObject based on the given typeInfo.
-     */
-    public static LazyObject<? extends ObjectInspector> createLazyObject(ObjectInspector oi) {
-        ObjectInspector.Category c = oi.getCategory();
-        switch (c) {
-            case PRIMITIVE:
-                return createLazyPrimitiveClass((PrimitiveObjectInspector) oi);
-            case MAP:
-                return new LazyMap((LazyMapObjectInspector) oi);
-            case LIST:
-                return new LazyArray((LazyListObjectInspector) oi);
-            case STRUCT: // check whether it is a top-level struct
-                if (oi instanceof LazyStructObjectInspector)
-                    return new LazyStruct((LazyStructObjectInspector) oi);
-                else
-                    return new LazyColumnar((LazyColumnarObjectInspector) oi);
-        }
+	/**
+	 * Create a hierarchical LazyObject based on the given typeInfo.
+	 */
+	public static LazyObject<? extends ObjectInspector> createLazyObject(
+			ObjectInspector oi) {
+		ObjectInspector.Category c = oi.getCategory();
+		switch (c) {
+		case PRIMITIVE:
+			return createLazyPrimitiveClass((PrimitiveObjectInspector) oi);
+		case MAP:
+			return new LazyMap((LazyMapObjectInspector) oi);
+		case LIST:
+			return new LazyArray((LazyListObjectInspector) oi);
+		case STRUCT: // check whether it is a top-level struct
+			if (oi instanceof LazyStructObjectInspector)
+				return new LazyStruct((LazyStructObjectInspector) oi);
+			else
+				return new LazyColumnar((LazyColumnarObjectInspector) oi);
+		default:
+			throw new RuntimeException("Hive LazySerDe Internal error.");
+		}
+	}
 
-        throw new RuntimeException("Hive LazySerDe Internal error.");
-    }
-
-    private LazyFactory() {
-        // prevent instantiation
-    }
+	private LazyFactory() {
+		// prevent instantiation
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java
index 26aa299..303cc67 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyFloat.java
@@ -25,28 +25,29 @@
  * LazyObject for storing a value of Double.
  * 
  */
-public class LazyFloat extends LazyPrimitive<LazyFloatObjectInspector, FloatWritable> {
+public class LazyFloat extends
+		LazyPrimitive<LazyFloatObjectInspector, FloatWritable> {
 
-    public LazyFloat(LazyFloatObjectInspector oi) {
-        super(oi);
-        data = new FloatWritable();
-    }
+	public LazyFloat(LazyFloatObjectInspector oi) {
+		super(oi);
+		data = new FloatWritable();
+	}
 
-    public LazyFloat(LazyFloat copy) {
-        super(copy);
-        data = new FloatWritable(copy.data.get());
-    }
+	public LazyFloat(LazyFloat copy) {
+		super(copy);
+		data = new FloatWritable(copy.data.get());
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
 
-        assert (4 == length);
-        data.set(Float.intBitsToFloat(LazyUtils.byteArrayToInt(bytes, start)));
-    }
+		assert (4 == length);
+		data.set(Float.intBitsToFloat(LazyUtils.byteArrayToInt(bytes, start)));
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java
index 5a0f0c1..c908c40 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyInteger.java
@@ -35,36 +35,38 @@
  * </p>
  * 
  */
-public class LazyInteger extends LazyPrimitive<LazyIntObjectInspector, IntWritable> {
+public class LazyInteger extends
+		LazyPrimitive<LazyIntObjectInspector, IntWritable> {
 
-    public LazyInteger(LazyIntObjectInspector oi) {
-        super(oi);
-        data = new IntWritable();
-    }
+	public LazyInteger(LazyIntObjectInspector oi) {
+		super(oi);
+		data = new IntWritable();
+	}
 
-    public LazyInteger(LazyInteger copy) {
-        super(copy);
-        data = new IntWritable(copy.data.get());
-    }
+	public LazyInteger(LazyInteger copy) {
+		super(copy);
+		data = new IntWritable(copy.data.get());
+	}
 
-    /**
-     * The reusable vInt for decoding the integer.
-     */
-    VInt vInt = new LazyUtils.VInt();
+	/**
+	 * The reusable vInt for decoding the integer.
+	 */
+	VInt vInt = new LazyUtils.VInt();
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
 
-        LazyUtils.readVInt(bytes, start, vInt);
-        assert (length == vInt.length);
-        if (length != vInt.length)
-            throw new IllegalStateException("parse int: length mismatch, expected " + vInt.length + " but get "
-                    + length);
-        data.set(vInt.value);
-    }
+		LazyUtils.readVInt(bytes, start, vInt);
+		assert (length == vInt.length);
+		if (length != vInt.length)
+			throw new IllegalStateException(
+					"parse int: length mismatch, expected " + vInt.length
+							+ " but get " + length);
+		data.set(vInt.value);
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java
index 2d02e53..38097e6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyLong.java
@@ -35,36 +35,37 @@
  * </p>
  * 
  */
-public class LazyLong extends LazyPrimitive<LazyLongObjectInspector, LongWritable> {
+public class LazyLong extends
+		LazyPrimitive<LazyLongObjectInspector, LongWritable> {
 
-    public LazyLong(LazyLongObjectInspector oi) {
-        super(oi);
-        data = new LongWritable();
-    }
+	public LazyLong(LazyLongObjectInspector oi) {
+		super(oi);
+		data = new LongWritable();
+	}
 
-    public LazyLong(LazyLong copy) {
-        super(copy);
-        data = new LongWritable(copy.data.get());
-    }
+	public LazyLong(LazyLong copy) {
+		super(copy);
+		data = new LongWritable(copy.data.get());
+	}
 
-    /**
-     * The reusable vLong for decoding the long.
-     */
-    VLong vLong = new LazyUtils.VLong();
+	/**
+	 * The reusable vLong for decoding the long.
+	 */
+	VLong vLong = new LazyUtils.VLong();
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
 
-        LazyUtils.readVLong(bytes, start, vLong);
-        assert (length == vLong.length);
-        if (length != vLong.length)
-            throw new IllegalStateException("parse long: length mismatch");
-        data.set(vLong.value);
-    }
+		LazyUtils.readVLong(bytes, start, vLong);
+		assert (length == vLong.length);
+		if (length != vLong.length)
+			throw new IllegalStateException("parse long: length mismatch");
+		data.set(vLong.value);
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java
index c84a1d2..56bc41b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyMap.java
@@ -46,286 +46,292 @@
  * 
  */
 
+@SuppressWarnings("rawtypes")
 public class LazyMap extends LazyNonPrimitive<LazyMapObjectInspector> {
 
-    private static Log LOG = LogFactory.getLog(LazyMap.class.getName());
+	private static Log LOG = LogFactory.getLog(LazyMap.class.getName());
 
-    /**
-     * Whether the data is already parsed or not.
-     */
-    boolean parsed;
+	/**
+	 * Whether the data is already parsed or not.
+	 */
+	boolean parsed;
 
-    /**
-     * The size of the map. Only valid when the data is parsed. -1 when the map
-     * is NULL.
-     */
-    int mapSize = 0;
+	/**
+	 * The size of the map. Only valid when the data is parsed. -1 when the map
+	 * is NULL.
+	 */
+	int mapSize = 0;
 
-    /**
-     * The beginning position and length of key[i] and value[i]. Only valid when
-     * the data is parsed.
-     */
-    int[] keyStart;
-    int[] keyLength;
-    int[] valueStart;
-    int[] valueLength;
-    /**
-     * Whether valueObjects[i]/keyObjects[i] is initialized or not.
-     */
-    boolean[] keyInited;
-    boolean[] valueInited;
+	/**
+	 * The beginning position and length of key[i] and value[i]. Only valid when
+	 * the data is parsed.
+	 */
+	int[] keyStart;
+	int[] keyLength;
+	int[] valueStart;
+	int[] valueLength;
+	/**
+	 * Whether valueObjects[i]/keyObjects[i] is initialized or not.
+	 */
+	boolean[] keyInited;
+	boolean[] valueInited;
 
-    /**
-     * Whether valueObjects[i]/keyObjects[i] is null or not This could not be
-     * inferred from the length of the object. In particular, a 0-length string
-     * is not null.
-     */
-    boolean[] keyIsNull;
-    boolean[] valueIsNull;
+	/**
+	 * Whether valueObjects[i]/keyObjects[i] is null or not This could not be
+	 * inferred from the length of the object. In particular, a 0-length string
+	 * is not null.
+	 */
+	boolean[] keyIsNull;
+	boolean[] valueIsNull;
 
-    /**
-     * The keys are stored in an array of LazyPrimitives.
-     */
-    LazyPrimitive<?, ?>[] keyObjects;
-    /**
-     * The values are stored in an array of LazyObjects. value[index] will start
-     * from KeyEnd[index] + 1, and ends before KeyStart[index+1] - 1.
-     */
-    LazyObject[] valueObjects;
+	/**
+	 * The keys are stored in an array of LazyPrimitives.
+	 */
+	LazyPrimitive<?, ?>[] keyObjects;
+	/**
+	 * The values are stored in an array of LazyObjects. value[index] will start
+	 * from KeyEnd[index] + 1, and ends before KeyStart[index+1] - 1.
+	 */
+	LazyObject[] valueObjects;
 
-    protected LazyMap(LazyMapObjectInspector oi) {
-        super(oi);
-    }
+	protected LazyMap(LazyMapObjectInspector oi) {
+		super(oi);
+	}
 
-    /**
-     * Set the row data for this LazyMap.
-     * 
-     * @see LazyObject#init(ByteArrayRef, int, int)
-     */
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        super.init(bytes, start, length);
-        parsed = false;
-    }
+	/**
+	 * Set the row data for this LazyMap.
+	 * 
+	 * @see LazyObject#init(ByteArrayRef, int, int)
+	 */
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		super.init(bytes, start, length);
+		parsed = false;
+	}
 
-    /**
-     * Adjust the size of arrays: keyStart, keyLength valueStart, valueLength
-     * keyInited, keyIsNull valueInited, valueIsNull.
-     */
-    protected void adjustArraySize(int newSize) {
-        if (keyStart == null || keyStart.length < newSize) {
-            keyStart = new int[newSize];
-            keyLength = new int[newSize];
-            valueStart = new int[newSize];
-            valueLength = new int[newSize];
-            keyInited = new boolean[newSize];
-            keyIsNull = new boolean[newSize];
-            valueInited = new boolean[newSize];
-            valueIsNull = new boolean[newSize];
-            keyObjects = new LazyPrimitive<?, ?>[newSize];
-            valueObjects = new LazyObject[newSize];
-        }
-    }
+	/**
+	 * Adjust the size of arrays: keyStart, keyLength valueStart, valueLength
+	 * keyInited, keyIsNull valueInited, valueIsNull.
+	 */
+	protected void adjustArraySize(int newSize) {
+		if (keyStart == null || keyStart.length < newSize) {
+			keyStart = new int[newSize];
+			keyLength = new int[newSize];
+			valueStart = new int[newSize];
+			valueLength = new int[newSize];
+			keyInited = new boolean[newSize];
+			keyIsNull = new boolean[newSize];
+			valueInited = new boolean[newSize];
+			valueIsNull = new boolean[newSize];
+			keyObjects = new LazyPrimitive<?, ?>[newSize];
+			valueObjects = new LazyObject[newSize];
+		}
+	}
 
-    boolean nullMapKey = false;
-    VInt vInt = new LazyUtils.VInt();
-    RecordInfo recordInfo = new LazyUtils.RecordInfo();
+	boolean nullMapKey = false;
+	VInt vInt = new LazyUtils.VInt();
+	RecordInfo recordInfo = new LazyUtils.RecordInfo();
 
-    /**
-     * Parse the byte[] and fill keyStart, keyLength, keyIsNull valueStart,
-     * valueLength and valueIsNull.
-     */
-    private void parse() {
+	/**
+	 * Parse the byte[] and fill keyStart, keyLength, keyIsNull valueStart,
+	 * valueLength and valueIsNull.
+	 */
+	private void parse() {
 
-        // get the VInt that represents the map size
-        LazyUtils.readVInt(bytes, start, vInt);
-        mapSize = vInt.value;
-        if (0 == mapSize) {
-            parsed = true;
-            return;
-        }
+		// get the VInt that represents the map size
+		LazyUtils.readVInt(bytes, start, vInt);
+		mapSize = vInt.value;
+		if (0 == mapSize) {
+			parsed = true;
+			return;
+		}
 
-        // adjust arrays
-        adjustArraySize(mapSize);
+		// adjust arrays
+		adjustArraySize(mapSize);
 
-        // find out the null-bytes
-        int mapByteStart = start + vInt.length;
-        int nullByteCur = mapByteStart;
-        int nullByteEnd = mapByteStart + (mapSize * 2 + 7) / 8;
-        int lastElementByteEnd = nullByteEnd;
+		// find out the null-bytes
+		int mapByteStart = start + vInt.length;
+		int nullByteCur = mapByteStart;
+		int nullByteEnd = mapByteStart + (mapSize * 2 + 7) / 8;
+		int lastElementByteEnd = nullByteEnd;
 
-        // parsing the keys and values one by one
-        for (int i = 0; i < mapSize; i++) {
-            // parse a key
-            keyIsNull[i] = true;
-            if ((bytes[nullByteCur] & (1 << ((i * 2) % 8))) != 0) {
-                keyIsNull[i] = false;
-                LazyUtils.checkObjectByteInfo(((MapObjectInspector) oi).getMapKeyObjectInspector(), bytes,
-                        lastElementByteEnd, recordInfo);
-                keyStart[i] = lastElementByteEnd + recordInfo.elementOffset;
-                keyLength[i] = recordInfo.elementSize;
-                lastElementByteEnd = keyStart[i] + keyLength[i];
-            } else if (!nullMapKey) {
-                nullMapKey = true;
-                LOG.warn("Null map key encountered! Ignoring similar problems.");
-            }
+		// parsing the keys and values one by one
+		for (int i = 0; i < mapSize; i++) {
+			// parse a key
+			keyIsNull[i] = true;
+			if ((bytes[nullByteCur] & (1 << ((i * 2) % 8))) != 0) {
+				keyIsNull[i] = false;
+				LazyUtils.checkObjectByteInfo(
+						((MapObjectInspector) oi).getMapKeyObjectInspector(),
+						bytes, lastElementByteEnd, recordInfo);
+				keyStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+				keyLength[i] = recordInfo.elementSize;
+				lastElementByteEnd = keyStart[i] + keyLength[i];
+			} else if (!nullMapKey) {
+				nullMapKey = true;
+				LOG.warn("Null map key encountered! Ignoring similar problems.");
+			}
 
-            // parse a value
-            valueIsNull[i] = true;
-            if ((bytes[nullByteCur] & (1 << ((i * 2 + 1) % 8))) != 0) {
-                valueIsNull[i] = false;
-                LazyUtils.checkObjectByteInfo(((MapObjectInspector) oi).getMapValueObjectInspector(), bytes,
-                        lastElementByteEnd, recordInfo);
-                valueStart[i] = lastElementByteEnd + recordInfo.elementOffset;
-                valueLength[i] = recordInfo.elementSize;
-                lastElementByteEnd = valueStart[i] + valueLength[i];
-            }
+			// parse a value
+			valueIsNull[i] = true;
+			if ((bytes[nullByteCur] & (1 << ((i * 2 + 1) % 8))) != 0) {
+				valueIsNull[i] = false;
+				LazyUtils.checkObjectByteInfo(
+						((MapObjectInspector) oi).getMapValueObjectInspector(),
+						bytes, lastElementByteEnd, recordInfo);
+				valueStart[i] = lastElementByteEnd + recordInfo.elementOffset;
+				valueLength[i] = recordInfo.elementSize;
+				lastElementByteEnd = valueStart[i] + valueLength[i];
+			}
 
-            // move onto the next null byte
-            if (3 == (i % 4)) {
-                nullByteCur++;
-            }
-        }
+			// move onto the next null byte
+			if (3 == (i % 4)) {
+				nullByteCur++;
+			}
+		}
 
-        Arrays.fill(keyInited, 0, mapSize, false);
-        Arrays.fill(valueInited, 0, mapSize, false);
-        parsed = true;
-    }
+		Arrays.fill(keyInited, 0, mapSize, false);
+		Arrays.fill(valueInited, 0, mapSize, false);
+		parsed = true;
+	}
 
-    /**
-     * Get the value object with the index without checking parsed.
-     * 
-     * @param index
-     *            The index into the array starting from 0
-     */
-    private LazyObject uncheckedGetValue(int index) {
-        if (valueIsNull[index]) {
-            return null;
-        }
-        if (!valueInited[index]) {
-            valueInited[index] = true;
-            if (valueObjects[index] == null) {
-                valueObjects[index] = LazyFactory.createLazyObject(((MapObjectInspector) oi)
-                        .getMapValueObjectInspector());
-            }
-            valueObjects[index].init(bytes, valueStart[index], valueLength[index]);
-        }
-        return valueObjects[index];
-    }
+	/**
+	 * Get the value object with the index without checking parsed.
+	 * 
+	 * @param index
+	 *            The index into the array starting from 0
+	 */
+	private LazyObject uncheckedGetValue(int index) {
+		if (valueIsNull[index]) {
+			return null;
+		}
+		if (!valueInited[index]) {
+			valueInited[index] = true;
+			if (valueObjects[index] == null) {
+				valueObjects[index] = LazyFactory
+						.createLazyObject(((MapObjectInspector) oi)
+								.getMapValueObjectInspector());
+			}
+			valueObjects[index].init(bytes, valueStart[index],
+					valueLength[index]);
+		}
+		return valueObjects[index];
+	}
 
-    /**
-     * Get the value in the map for the key.
-     * 
-     * If there are multiple matches (which is possible in the serialized
-     * format), only the first one is returned.
-     * 
-     * The most efficient way to get the value for the key is to serialize the
-     * key and then try to find it in the array. We do linear search because in
-     * most cases, user only wants to get one or two values out of the map, and
-     * the cost of building up a HashMap is substantially higher.
-     * 
-     * @param key
-     *            The key object that we are looking for.
-     * @return The corresponding value object, or NULL if not found
-     */
-    public Object getMapValueElement(Object key) {
-        if (!parsed) {
-            parse();
-        }
-        // search for the key
-        for (int i = 0; i < mapSize; i++) {
-            LazyPrimitive<?, ?> lazyKeyI = uncheckedGetKey(i);
-            if (lazyKeyI == null) {
-                continue;
-            }
-            // getWritableObject() will convert LazyPrimitive to actual
-            // primitive
-            // writable objects.
-            Object keyI = lazyKeyI.getWritableObject();
-            if (keyI == null) {
-                continue;
-            }
-            if (keyI.equals(key)) {
-                // Got a match, return the value
-                LazyObject v = uncheckedGetValue(i);
-                return v == null ? v : v.getObject();
-            }
-        }
-        return null;
-    }
+	/**
+	 * Get the value in the map for the key.
+	 * 
+	 * If there are multiple matches (which is possible in the serialized
+	 * format), only the first one is returned.
+	 * 
+	 * The most efficient way to get the value for the key is to serialize the
+	 * key and then try to find it in the array. We do linear search because in
+	 * most cases, user only wants to get one or two values out of the map, and
+	 * the cost of building up a HashMap is substantially higher.
+	 * 
+	 * @param key
+	 *            The key object that we are looking for.
+	 * @return The corresponding value object, or NULL if not found
+	 */
+	public Object getMapValueElement(Object key) {
+		if (!parsed) {
+			parse();
+		}
+		// search for the key
+		for (int i = 0; i < mapSize; i++) {
+			LazyPrimitive<?, ?> lazyKeyI = uncheckedGetKey(i);
+			if (lazyKeyI == null) {
+				continue;
+			}
+			// getWritableObject() will convert LazyPrimitive to actual
+			// primitive
+			// writable objects.
+			Object keyI = lazyKeyI.getWritableObject();
+			if (keyI == null) {
+				continue;
+			}
+			if (keyI.equals(key)) {
+				// Got a match, return the value
+				LazyObject v = uncheckedGetValue(i);
+				return v == null ? v : v.getObject();
+			}
+		}
+		return null;
+	}
 
-    /**
-     * Get the key object with the index without checking parsed.
-     * 
-     * @param index
-     *            The index into the array starting from 0
-     */
-    private LazyPrimitive<?, ?> uncheckedGetKey(int index) {
-        if (keyIsNull[index]) {
-            return null;
-        }
-        if (!keyInited[index]) {
-            keyInited[index] = true;
-            if (keyObjects[index] == null) {
-                // Keys are always primitive
-                keyObjects[index] = LazyFactory
-                        .createLazyPrimitiveClass((PrimitiveObjectInspector) ((MapObjectInspector) oi)
-                                .getMapKeyObjectInspector());
-            }
-            keyObjects[index].init(bytes, keyStart[index], keyLength[index]);
-        }
-        return keyObjects[index];
-    }
+	/**
+	 * Get the key object with the index without checking parsed.
+	 * 
+	 * @param index
+	 *            The index into the array starting from 0
+	 */
+	private LazyPrimitive<?, ?> uncheckedGetKey(int index) {
+		if (keyIsNull[index]) {
+			return null;
+		}
+		if (!keyInited[index]) {
+			keyInited[index] = true;
+			if (keyObjects[index] == null) {
+				// Keys are always primitive
+				keyObjects[index] = LazyFactory
+						.createLazyPrimitiveClass((PrimitiveObjectInspector) ((MapObjectInspector) oi)
+								.getMapKeyObjectInspector());
+			}
+			keyObjects[index].init(bytes, keyStart[index], keyLength[index]);
+		}
+		return keyObjects[index];
+	}
 
-    /**
-     * cachedMap is reused for different calls to getMap(). But each LazyMap has
-     * a separate cachedMap so we won't overwrite the data by accident.
-     */
-    LinkedHashMap<Object, Object> cachedMap;
+	/**
+	 * cachedMap is reused for different calls to getMap(). But each LazyMap has
+	 * a separate cachedMap so we won't overwrite the data by accident.
+	 */
+	LinkedHashMap<Object, Object> cachedMap;
 
-    /**
-     * Return the map object representing this LazyMap. Note that the keyObjects
-     * will be Writable primitive objects.
-     * 
-     * @return the map object
-     */
-    public Map<Object, Object> getMap() {
-        if (!parsed) {
-            parse();
-        }
-        if (cachedMap == null) {
-            // Use LinkedHashMap to provide deterministic order
-            cachedMap = new LinkedHashMap<Object, Object>();
-        } else {
-            cachedMap.clear();
-        }
+	/**
+	 * Return the map object representing this LazyMap. Note that the keyObjects
+	 * will be Writable primitive objects.
+	 * 
+	 * @return the map object
+	 */
+	public Map<Object, Object> getMap() {
+		if (!parsed) {
+			parse();
+		}
+		if (cachedMap == null) {
+			// Use LinkedHashMap to provide deterministic order
+			cachedMap = new LinkedHashMap<Object, Object>();
+		} else {
+			cachedMap.clear();
+		}
 
-        // go through each element of the map
-        for (int i = 0; i < mapSize; i++) {
-            LazyPrimitive<?, ?> lazyKey = uncheckedGetKey(i);
-            if (lazyKey == null) {
-                continue;
-            }
-            Object key = lazyKey.getObject();
-            // do not overwrite if there are duplicate keys
-            if (key != null && !cachedMap.containsKey(key)) {
-                LazyObject lazyValue = uncheckedGetValue(i);
-                Object value = (lazyValue == null ? null : lazyValue.getObject());
-                cachedMap.put(key, value);
-            }
-        }
-        return cachedMap;
-    }
+		// go through each element of the map
+		for (int i = 0; i < mapSize; i++) {
+			LazyPrimitive<?, ?> lazyKey = uncheckedGetKey(i);
+			if (lazyKey == null) {
+				continue;
+			}
+			Object key = lazyKey.getObject();
+			// do not overwrite if there are duplicate keys
+			if (key != null && !cachedMap.containsKey(key)) {
+				LazyObject lazyValue = uncheckedGetValue(i);
+				Object value = (lazyValue == null ? null : lazyValue
+						.getObject());
+				cachedMap.put(key, value);
+			}
+		}
+		return cachedMap;
+	}
 
-    /**
-     * Get the size of the map represented by this LazyMap.
-     * 
-     * @return The size of the map
-     */
-    public int getMapSize() {
-        if (!parsed) {
-            parse();
-        }
-        return mapSize;
-    }
+	/**
+	 * Get the size of the map represented by this LazyMap.
+	 * 
+	 * @return The size of the map
+	 */
+	public int getMapSize() {
+		if (!parsed) {
+			parse();
+		}
+		return mapSize;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java
index f7ae1e3..b151f2d 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyNonPrimitive.java
@@ -24,54 +24,55 @@
 /**
  * LazyPrimitive stores a primitive Object in a LazyObject.
  */
-public abstract class LazyNonPrimitive<OI extends ObjectInspector> extends LazyObject<OI> {
+public abstract class LazyNonPrimitive<OI extends ObjectInspector> extends
+		LazyObject<OI> {
 
-    protected byte[] bytes;
-    protected int start;
-    protected int length;
+	protected byte[] bytes;
+	protected int start;
+	protected int length;
 
-    /**
-     * Create a LazyNonPrimitive object with the specified ObjectInspector.
-     * 
-     * @param oi
-     *            The ObjectInspector would have to have a hierarchy of
-     *            LazyObjectInspectors with the leaf nodes being
-     *            WritableObjectInspectors. It's used both for accessing the
-     *            type hierarchy of the complex object, as well as getting meta
-     *            information (separator, nullSequence, etc) when parsing the
-     *            lazy object.
-     */
-    protected LazyNonPrimitive(OI oi) {
-        super(oi);
-        bytes = null;
-        start = 0;
-        length = 0;
-    }
+	/**
+	 * Create a LazyNonPrimitive object with the specified ObjectInspector.
+	 * 
+	 * @param oi
+	 *            The ObjectInspector would have to have a hierarchy of
+	 *            LazyObjectInspectors with the leaf nodes being
+	 *            WritableObjectInspectors. It's used both for accessing the
+	 *            type hierarchy of the complex object, as well as getting meta
+	 *            information (separator, nullSequence, etc) when parsing the
+	 *            lazy object.
+	 */
+	protected LazyNonPrimitive(OI oi) {
+		super(oi);
+		bytes = null;
+		start = 0;
+		length = 0;
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (bytes == null) {
-            throw new RuntimeException("bytes cannot be null!");
-        }
-        this.bytes = bytes;
-        this.start = start;
-        this.length = length;
-        assert start >= 0;
-        assert start + length <= bytes.length;
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (bytes == null) {
+			throw new RuntimeException("bytes cannot be null!");
+		}
+		this.bytes = bytes;
+		this.start = start;
+		this.length = length;
+		assert start >= 0;
+		assert start + length <= bytes.length;
+	}
 
-    @Override
-    public Object getObject() {
-        return this;
-    }
+	@Override
+	public Object getObject() {
+		return this;
+	}
 
-    @Override
-    public int hashCode() {
-        return LazyUtils.hashBytes(bytes, start, length);
-    }
+	@Override
+	public int hashCode() {
+		return LazyUtils.hashBytes(bytes, start, length);
+	}
 
-    @Override
-    public void init(IFrameTupleReference tuple) {
-    }
+	@Override
+	public void init(IFrameTupleReference tuple) {
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java
index d71817b..9aaaa88 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyObject.java
@@ -29,50 +29,50 @@
  */
 public abstract class LazyObject<OI extends ObjectInspector> {
 
-    OI oi;
+	OI oi;
 
-    /**
-     * Create a LazyObject.
-     * 
-     * @param oi
-     *            Derived classes can access meta information about this Lazy
-     *            Object (e.g, separator, nullSequence, escaper) from it.
-     */
-    protected LazyObject(OI oi) {
-        this.oi = oi;
-    }
+	/**
+	 * Create a LazyObject.
+	 * 
+	 * @param oi
+	 *            Derived classes can access meta information about this Lazy
+	 *            Object (e.g, separator, nullSequence, escaper) from it.
+	 */
+	protected LazyObject(OI oi) {
+		this.oi = oi;
+	}
 
-    /**
-     * Set the data for this LazyObject. We take ByteArrayRef instead of byte[]
-     * so that we will be able to drop the reference to byte[] by a single
-     * assignment. The ByteArrayRef object can be reused across multiple rows.
-     * 
-     * @param bytes
-     *            The wrapper of the byte[].
-     * @param start
-     *            The start position inside the bytes.
-     * @param length
-     *            The length of the data, starting from "start"
-     * @see ByteArrayRef
-     */
-    public abstract void init(byte[] bytes, int start, int length);
+	/**
+	 * Set the data for this LazyObject. We take ByteArrayRef instead of byte[]
+	 * so that we will be able to drop the reference to byte[] by a single
+	 * assignment. The ByteArrayRef object can be reused across multiple rows.
+	 * 
+	 * @param bytes
+	 *            The wrapper of the byte[].
+	 * @param start
+	 *            The start position inside the bytes.
+	 * @param length
+	 *            The length of the data, starting from "start"
+	 * @see ByteArrayRef
+	 */
+	public abstract void init(byte[] bytes, int start, int length);
 
-    public abstract void init(IFrameTupleReference tuple);
+	public abstract void init(IFrameTupleReference tuple);
 
-    /**
-     * If the LazyObject is a primitive Object, then deserialize it and return
-     * the actual primitive Object. Otherwise (array, map, struct), return this.
-     */
-    public abstract Object getObject();
+	/**
+	 * If the LazyObject is a primitive Object, then deserialize it and return
+	 * the actual primitive Object. Otherwise (array, map, struct), return this.
+	 */
+	public abstract Object getObject();
 
-    @Override
-    public abstract int hashCode();
+	@Override
+	public abstract int hashCode();
 
-    protected OI getInspector() {
-        return oi;
-    }
+	protected OI getInspector() {
+		return oi;
+	}
 
-    protected void setInspector(OI oi) {
-        this.oi = oi;
-    }
+	protected void setInspector(OI oi) {
+		this.oi = oi;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java
index 8139c65..888e5b2 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyPrimitive.java
@@ -25,45 +25,46 @@
 /**
  * LazyPrimitive stores a primitive Object in a LazyObject.
  */
-public abstract class LazyPrimitive<OI extends ObjectInspector, T extends Writable> extends LazyObject<OI> {
+public abstract class LazyPrimitive<OI extends ObjectInspector, T extends Writable>
+		extends LazyObject<OI> {
 
-    LazyPrimitive(OI oi) {
-        super(oi);
-    }
+	LazyPrimitive(OI oi) {
+		super(oi);
+	}
 
-    LazyPrimitive(LazyPrimitive<OI, T> copy) {
-        super(copy.oi);
-        isNull = copy.isNull;
-    }
+	LazyPrimitive(LazyPrimitive<OI, T> copy) {
+		super(copy.oi);
+		isNull = copy.isNull;
+	}
 
-    T data;
-    boolean isNull = false;
+	T data;
+	boolean isNull = false;
 
-    /**
-     * Returns the primitive object represented by this LazyObject. This is
-     * useful because it can make sure we have "null" for null objects.
-     */
-    @Override
-    public Object getObject() {
-        return isNull ? null : this;
-    }
+	/**
+	 * Returns the primitive object represented by this LazyObject. This is
+	 * useful because it can make sure we have "null" for null objects.
+	 */
+	@Override
+	public Object getObject() {
+		return isNull ? null : this;
+	}
 
-    public T getWritableObject() {
-        return isNull ? null : data;
-    }
+	public T getWritableObject() {
+		return isNull ? null : data;
+	}
 
-    @Override
-    public String toString() {
-        return isNull ? "null" : data.toString();
-    }
+	@Override
+	public String toString() {
+		return isNull ? "null" : data.toString();
+	}
 
-    @Override
-    public int hashCode() {
-        return isNull ? 0 : data.hashCode();
-    }
+	@Override
+	public int hashCode() {
+		return isNull ? 0 : data.hashCode();
+	}
 
-    @Override
-    public void init(IFrameTupleReference tuple) {
-    }
+	@Override
+	public void init(IFrameTupleReference tuple) {
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
index a5d4b3b..4d0dff6 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazySerDe.java
@@ -62,398 +62,416 @@
  */
 public class LazySerDe implements SerDe {
 
-    public static final Log LOG = LogFactory.getLog(LazySerDe.class.getName());
+	public static final Log LOG = LogFactory.getLog(LazySerDe.class.getName());
 
-    public LazySerDe() {
-    }
+	public LazySerDe() {
+	}
 
-    List<String> columnNames;
-    List<TypeInfo> columnTypes;
+	List<String> columnNames;
+	List<TypeInfo> columnTypes;
 
-    TypeInfo rowTypeInfo;
-    ObjectInspector cachedObjectInspector;
+	TypeInfo rowTypeInfo;
+	ObjectInspector cachedObjectInspector;
 
-    // The object for storing row data
-    LazyColumnar cachedLazyStruct;
+	// The object for storing row data
+	LazyColumnar cachedLazyStruct;
 
-    /**
-     * Initialize the SerDe with configuration and table information.
-     */
-    @Override
-    public void initialize(Configuration conf, Properties tbl) throws SerDeException {
-        // Get column names and types
-        String columnNameProperty = tbl.getProperty(Constants.LIST_COLUMNS);
-        String columnTypeProperty = tbl.getProperty(Constants.LIST_COLUMN_TYPES);
-        if (columnNameProperty.length() == 0) {
-            columnNames = new ArrayList<String>();
-        } else {
-            columnNames = Arrays.asList(columnNameProperty.split(","));
-        }
-        if (columnTypeProperty.length() == 0) {
-            columnTypes = new ArrayList<TypeInfo>();
-        } else {
-            columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
-        }
-        assert (columnNames.size() == columnTypes.size());
-        // Create row related objects
-        rowTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
-        // Create the object inspector and the lazy binary struct object
-        cachedObjectInspector = LazyUtils.getLazyObjectInspectorFromTypeInfo(rowTypeInfo, true);
-        cachedLazyStruct = (LazyColumnar) LazyFactory.createLazyObject(cachedObjectInspector);
-        // output debug info
-        LOG.debug("LazySerDe initialized with: columnNames=" + columnNames + " columnTypes=" + columnTypes);
-    }
+	/**
+	 * Initialize the SerDe with configuration and table information.
+	 */
+	@Override
+	public void initialize(Configuration conf, Properties tbl)
+			throws SerDeException {
+		// Get column names and types
+		String columnNameProperty = tbl.getProperty(Constants.LIST_COLUMNS);
+		String columnTypeProperty = tbl
+				.getProperty(Constants.LIST_COLUMN_TYPES);
+		if (columnNameProperty.length() == 0) {
+			columnNames = new ArrayList<String>();
+		} else {
+			columnNames = Arrays.asList(columnNameProperty.split(","));
+		}
+		if (columnTypeProperty.length() == 0) {
+			columnTypes = new ArrayList<TypeInfo>();
+		} else {
+			columnTypes = TypeInfoUtils
+					.getTypeInfosFromTypeString(columnTypeProperty);
+		}
+		assert (columnNames.size() == columnTypes.size());
+		// Create row related objects
+		rowTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNames,
+				columnTypes);
+		// Create the object inspector and the lazy binary struct object
+		cachedObjectInspector = LazyUtils.getLazyObjectInspectorFromTypeInfo(
+				rowTypeInfo, true);
+		cachedLazyStruct = (LazyColumnar) LazyFactory
+				.createLazyObject(cachedObjectInspector);
+		// output debug info
+		LOG.debug("LazySerDe initialized with: columnNames=" + columnNames
+				+ " columnTypes=" + columnTypes);
+	}
 
-    /**
-     * Returns the ObjectInspector for the row.
-     */
-    @Override
-    public ObjectInspector getObjectInspector() throws SerDeException {
-        return cachedObjectInspector;
-    }
+	/**
+	 * Returns the ObjectInspector for the row.
+	 */
+	@Override
+	public ObjectInspector getObjectInspector() throws SerDeException {
+		return cachedObjectInspector;
+	}
 
-    /**
-     * Returns the Writable Class after serialization.
-     */
-    @Override
-    public Class<? extends Writable> getSerializedClass() {
-        return BytesWritable.class;
-    }
+	/**
+	 * Returns the Writable Class after serialization.
+	 */
+	@Override
+	public Class<? extends Writable> getSerializedClass() {
+		return BytesWritable.class;
+	}
 
-    // The wrapper for byte array
-    ByteArrayRef byteArrayRef;
+	// The wrapper for byte array
+	ByteArrayRef byteArrayRef;
 
-    /**
-     * Deserialize a table record to a Lazy struct.
-     */
-    @Override
-    public Object deserialize(Writable field) throws SerDeException {
-        if (byteArrayRef == null) {
-            byteArrayRef = new ByteArrayRef();
-        }
-        if (field instanceof BytesWritable) {
-            BytesWritable b = (BytesWritable) field;
-            if (b.getSize() == 0) {
-                return null;
-            }
-            // For backward-compatibility with hadoop 0.17
-            byteArrayRef.setData(b.get());
-            cachedLazyStruct.init(byteArrayRef.getData(), 0, b.getSize());
-        } else if (field instanceof Text) {
-            Text t = (Text) field;
-            if (t.getLength() == 0) {
-                return null;
-            }
-            byteArrayRef.setData(t.getBytes());
-            cachedLazyStruct.init(byteArrayRef.getData(), 0, t.getLength());
-        } else {
-            throw new SerDeException(getClass().toString() + ": expects either BytesWritable or Text object!");
-        }
-        return cachedLazyStruct;
-    }
+	/**
+	 * Deserialize a table record to a Lazy struct.
+	 */
+	@SuppressWarnings("deprecation")
+	@Override
+	public Object deserialize(Writable field) throws SerDeException {
+		if (byteArrayRef == null) {
+			byteArrayRef = new ByteArrayRef();
+		}
+		if (field instanceof BytesWritable) {
+			BytesWritable b = (BytesWritable) field;
+			if (b.getSize() == 0) {
+				return null;
+			}
+			// For backward-compatibility with hadoop 0.17
+			byteArrayRef.setData(b.get());
+			cachedLazyStruct.init(byteArrayRef.getData(), 0, b.getSize());
+		} else if (field instanceof Text) {
+			Text t = (Text) field;
+			if (t.getLength() == 0) {
+				return null;
+			}
+			byteArrayRef.setData(t.getBytes());
+			cachedLazyStruct.init(byteArrayRef.getData(), 0, t.getLength());
+		} else {
+			throw new SerDeException(getClass().toString()
+					+ ": expects either BytesWritable or Text object!");
+		}
+		return cachedLazyStruct;
+	}
 
-    /**
-     * The reusable output buffer and serialize byte buffer.
-     */
-    BytesWritable serializeBytesWritable = new BytesWritable();
-    ByteStream.Output serializeByteStream = new ByteStream.Output();
+	/**
+	 * The reusable output buffer and serialize byte buffer.
+	 */
+	BytesWritable serializeBytesWritable = new BytesWritable();
+	ByteStream.Output serializeByteStream = new ByteStream.Output();
 
-    /**
-     * Serialize an object to a byte buffer in a binary compact way.
-     */
-    @Override
-    public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException {
-        // make sure it is a struct record or not
-        serializeByteStream.reset();
+	/**
+	 * Serialize an object to a byte buffer in a binary compact way.
+	 */
+	@Override
+	public Writable serialize(Object obj, ObjectInspector objInspector)
+			throws SerDeException {
+		// make sure it is a struct record or not
+		serializeByteStream.reset();
 
-        if (objInspector.getCategory() != Category.STRUCT) {
-            // serialize the primitive object
-            serialize(serializeByteStream, obj, objInspector);
-        } else {
-            // serialize the row as a struct
-            serializeStruct(serializeByteStream, obj, (StructObjectInspector) objInspector);
-        }
-        // return the serialized bytes
-        serializeBytesWritable.set(serializeByteStream.getData(), 0, serializeByteStream.getCount());
-        return serializeBytesWritable;
-    }
+		if (objInspector.getCategory() != Category.STRUCT) {
+			// serialize the primitive object
+			serialize(serializeByteStream, obj, objInspector);
+		} else {
+			// serialize the row as a struct
+			serializeStruct(serializeByteStream, obj,
+					(StructObjectInspector) objInspector);
+		}
+		// return the serialized bytes
+		serializeBytesWritable.set(serializeByteStream.getData(), 0,
+				serializeByteStream.getCount());
+		return serializeBytesWritable;
+	}
 
-    boolean nullMapKey = false;
+	boolean nullMapKey = false;
 
-    /**
-     * Serialize a struct object without writing the byte size. This function is
-     * shared by both row serialization and struct serialization.
-     * 
-     * @param byteStream
-     *            the byte stream storing the serialization data
-     * @param obj
-     *            the struct object to serialize
-     * @param objInspector
-     *            the struct object inspector
-     */
-    private void serializeStruct(Output byteStream, Object obj, StructObjectInspector soi) {
-        // do nothing for null struct
-        if (null == obj) {
-            return;
-        }
-        /*
-         * Interleave serializing one null byte and 8 struct fields in each
-         * round, in order to support data deserialization with different table
-         * schemas
-         */
-        List<? extends StructField> fields = soi.getAllStructFieldRefs();
-        int size = fields.size();
-        int lasti = 0;
-        byte nullByte = 0;
-        for (int i = 0; i < size; i++) {
-            // set bit to 1 if a field is not null
-            if (null != soi.getStructFieldData(obj, fields.get(i))) {
-                nullByte |= 1 << (i % 8);
-            }
-            // write the null byte every eight elements or
-            // if this is the last element and serialize the
-            // corresponding 8 struct fields at the same time
-            if (7 == i % 8 || i == size - 1) {
-                serializeByteStream.write(nullByte);
-                for (int j = lasti; j <= i; j++) {
-                    serialize(serializeByteStream, soi.getStructFieldData(obj, fields.get(j)), fields.get(j)
-                            .getFieldObjectInspector());
-                }
-                lasti = i + 1;
-                nullByte = 0;
-            }
-        }
-    }
+	/**
+	 * Serialize a struct object without writing the byte size. This function is
+	 * shared by both row serialization and struct serialization.
+	 * 
+	 * @param byteStream
+	 *            the byte stream storing the serialization data
+	 * @param obj
+	 *            the struct object to serialize
+	 * @param objInspector
+	 *            the struct object inspector
+	 */
+	private void serializeStruct(Output byteStream, Object obj,
+			StructObjectInspector soi) {
+		// do nothing for null struct
+		if (null == obj) {
+			return;
+		}
+		/*
+		 * Interleave serializing one null byte and 8 struct fields in each
+		 * round, in order to support data deserialization with different table
+		 * schemas
+		 */
+		List<? extends StructField> fields = soi.getAllStructFieldRefs();
+		int size = fields.size();
+		int lasti = 0;
+		byte nullByte = 0;
+		for (int i = 0; i < size; i++) {
+			// set bit to 1 if a field is not null
+			if (null != soi.getStructFieldData(obj, fields.get(i))) {
+				nullByte |= 1 << (i % 8);
+			}
+			// write the null byte every eight elements or
+			// if this is the last element and serialize the
+			// corresponding 8 struct fields at the same time
+			if (7 == i % 8 || i == size - 1) {
+				serializeByteStream.write(nullByte);
+				for (int j = lasti; j <= i; j++) {
+					serialize(serializeByteStream, soi.getStructFieldData(obj,
+							fields.get(j)), fields.get(j)
+							.getFieldObjectInspector());
+				}
+				lasti = i + 1;
+				nullByte = 0;
+			}
+		}
+	}
 
-    /**
-     * A recursive function that serialize an object to a byte buffer based on
-     * its object inspector.
-     * 
-     * @param byteStream
-     *            the byte stream storing the serialization data
-     * @param obj
-     *            the object to serialize
-     * @param objInspector
-     *            the object inspector
-     */
-    private void serialize(Output byteStream, Object obj, ObjectInspector objInspector) {
+	/**
+	 * A recursive function that serialize an object to a byte buffer based on
+	 * its object inspector.
+	 * 
+	 * @param byteStream
+	 *            the byte stream storing the serialization data
+	 * @param obj
+	 *            the object to serialize
+	 * @param objInspector
+	 *            the object inspector
+	 */
+	private void serialize(Output byteStream, Object obj,
+			ObjectInspector objInspector) {
 
-        // do nothing for null object
-        if (null == obj) {
-            return;
-        }
+		// do nothing for null object
+		if (null == obj) {
+			return;
+		}
 
-        switch (objInspector.getCategory()) {
-            case PRIMITIVE: {
-                PrimitiveObjectInspector poi = (PrimitiveObjectInspector) objInspector;
-                switch (poi.getPrimitiveCategory()) {
-                    case VOID: {
-                        return;
-                    }
-                    case BOOLEAN: {
-                        boolean v = ((BooleanObjectInspector) poi).get(obj);
-                        byteStream.write((byte) (v ? 1 : 0));
-                        return;
-                    }
-                    case BYTE: {
-                        ByteObjectInspector boi = (ByteObjectInspector) poi;
-                        byte v = boi.get(obj);
-                        byteStream.write(v);
-                        return;
-                    }
-                    case SHORT: {
-                        ShortObjectInspector spoi = (ShortObjectInspector) poi;
-                        short v = spoi.get(obj);
-                        byteStream.write((byte) (v >> 8));
-                        byteStream.write((byte) (v));
-                        return;
-                    }
-                    case INT: {
-                        IntObjectInspector ioi = (IntObjectInspector) poi;
-                        int v = ioi.get(obj);
-                        LazyUtils.writeVInt(byteStream, v);
-                        return;
-                    }
-                    case LONG: {
-                        LongObjectInspector loi = (LongObjectInspector) poi;
-                        long v = loi.get(obj);
-                        LazyUtils.writeVLong(byteStream, v);
-                        return;
-                    }
-                    case FLOAT: {
-                        FloatObjectInspector foi = (FloatObjectInspector) poi;
-                        int v = Float.floatToIntBits(foi.get(obj));
-                        byteStream.write((byte) (v >> 24));
-                        byteStream.write((byte) (v >> 16));
-                        byteStream.write((byte) (v >> 8));
-                        byteStream.write((byte) (v));
-                        return;
-                    }
-                    case DOUBLE: {
-                        DoubleObjectInspector doi = (DoubleObjectInspector) poi;
-                        long v = Double.doubleToLongBits(doi.get(obj));
-                        byteStream.write((byte) (v >> 56));
-                        byteStream.write((byte) (v >> 48));
-                        byteStream.write((byte) (v >> 40));
-                        byteStream.write((byte) (v >> 32));
-                        byteStream.write((byte) (v >> 24));
-                        byteStream.write((byte) (v >> 16));
-                        byteStream.write((byte) (v >> 8));
-                        byteStream.write((byte) (v));
-                        return;
-                    }
-                    case STRING: {
-                        StringObjectInspector soi = (StringObjectInspector) poi;
-                        Text t = soi.getPrimitiveWritableObject(obj);
-                        /* write byte size of the string which is a vint */
-                        int length = t.getLength();
-                        LazyUtils.writeVInt(byteStream, length);
-                        /* write string itself */
-                        byte[] data = t.getBytes();
-                        byteStream.write(data, 0, length);
-                        return;
-                    }
-                    default: {
-                        throw new RuntimeException("Unrecognized type: " + poi.getPrimitiveCategory());
-                    }
-                }
-            }
-            case LIST: {
-                ListObjectInspector loi = (ListObjectInspector) objInspector;
-                ObjectInspector eoi = loi.getListElementObjectInspector();
+		switch (objInspector.getCategory()) {
+		case PRIMITIVE: {
+			PrimitiveObjectInspector poi = (PrimitiveObjectInspector) objInspector;
+			switch (poi.getPrimitiveCategory()) {
+			case VOID: {
+				return;
+			}
+			case BOOLEAN: {
+				boolean v = ((BooleanObjectInspector) poi).get(obj);
+				byteStream.write((byte) (v ? 1 : 0));
+				return;
+			}
+			case BYTE: {
+				ByteObjectInspector boi = (ByteObjectInspector) poi;
+				byte v = boi.get(obj);
+				byteStream.write(v);
+				return;
+			}
+			case SHORT: {
+				ShortObjectInspector spoi = (ShortObjectInspector) poi;
+				short v = spoi.get(obj);
+				byteStream.write((byte) (v >> 8));
+				byteStream.write((byte) (v));
+				return;
+			}
+			case INT: {
+				IntObjectInspector ioi = (IntObjectInspector) poi;
+				int v = ioi.get(obj);
+				LazyUtils.writeVInt(byteStream, v);
+				return;
+			}
+			case LONG: {
+				LongObjectInspector loi = (LongObjectInspector) poi;
+				long v = loi.get(obj);
+				LazyUtils.writeVLong(byteStream, v);
+				return;
+			}
+			case FLOAT: {
+				FloatObjectInspector foi = (FloatObjectInspector) poi;
+				int v = Float.floatToIntBits(foi.get(obj));
+				byteStream.write((byte) (v >> 24));
+				byteStream.write((byte) (v >> 16));
+				byteStream.write((byte) (v >> 8));
+				byteStream.write((byte) (v));
+				return;
+			}
+			case DOUBLE: {
+				DoubleObjectInspector doi = (DoubleObjectInspector) poi;
+				long v = Double.doubleToLongBits(doi.get(obj));
+				byteStream.write((byte) (v >> 56));
+				byteStream.write((byte) (v >> 48));
+				byteStream.write((byte) (v >> 40));
+				byteStream.write((byte) (v >> 32));
+				byteStream.write((byte) (v >> 24));
+				byteStream.write((byte) (v >> 16));
+				byteStream.write((byte) (v >> 8));
+				byteStream.write((byte) (v));
+				return;
+			}
+			case STRING: {
+				StringObjectInspector soi = (StringObjectInspector) poi;
+				Text t = soi.getPrimitiveWritableObject(obj);
+				/* write byte size of the string which is a vint */
+				int length = t.getLength();
+				LazyUtils.writeVInt(byteStream, length);
+				/* write string itself */
+				byte[] data = t.getBytes();
+				byteStream.write(data, 0, length);
+				return;
+			}
+			default: {
+				throw new RuntimeException("Unrecognized type: "
+						+ poi.getPrimitiveCategory());
+			}
+			}
+		}
+		case LIST: {
+			ListObjectInspector loi = (ListObjectInspector) objInspector;
+			ObjectInspector eoi = loi.getListElementObjectInspector();
 
-                // 1/ reserve spaces for the byte size of the list
-                // which is a integer and takes four bytes
-                int byteSizeStart = byteStream.getCount();
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                int listStart = byteStream.getCount();
+			// 1/ reserve spaces for the byte size of the list
+			// which is a integer and takes four bytes
+			int byteSizeStart = byteStream.getCount();
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			int listStart = byteStream.getCount();
 
-                // 2/ write the size of the list as a VInt
-                int size = loi.getListLength(obj);
-                LazyUtils.writeVInt(byteStream, size);
+			// 2/ write the size of the list as a VInt
+			int size = loi.getListLength(obj);
+			LazyUtils.writeVInt(byteStream, size);
 
-                // 3/ write the null bytes
-                byte nullByte = 0;
-                for (int eid = 0; eid < size; eid++) {
-                    // set the bit to 1 if an element is not null
-                    if (null != loi.getListElement(obj, eid)) {
-                        nullByte |= 1 << (eid % 8);
-                    }
-                    // store the byte every eight elements or
-                    // if this is the last element
-                    if (7 == eid % 8 || eid == size - 1) {
-                        byteStream.write(nullByte);
-                        nullByte = 0;
-                    }
-                }
+			// 3/ write the null bytes
+			byte nullByte = 0;
+			for (int eid = 0; eid < size; eid++) {
+				// set the bit to 1 if an element is not null
+				if (null != loi.getListElement(obj, eid)) {
+					nullByte |= 1 << (eid % 8);
+				}
+				// store the byte every eight elements or
+				// if this is the last element
+				if (7 == eid % 8 || eid == size - 1) {
+					byteStream.write(nullByte);
+					nullByte = 0;
+				}
+			}
 
-                // 4/ write element by element from the list
-                for (int eid = 0; eid < size; eid++) {
-                    serialize(byteStream, loi.getListElement(obj, eid), eoi);
-                }
+			// 4/ write element by element from the list
+			for (int eid = 0; eid < size; eid++) {
+				serialize(byteStream, loi.getListElement(obj, eid), eoi);
+			}
 
-                // 5/ update the list byte size
-                int listEnd = byteStream.getCount();
-                int listSize = listEnd - listStart;
-                byte[] bytes = byteStream.getData();
-                bytes[byteSizeStart] = (byte) (listSize >> 24);
-                bytes[byteSizeStart + 1] = (byte) (listSize >> 16);
-                bytes[byteSizeStart + 2] = (byte) (listSize >> 8);
-                bytes[byteSizeStart + 3] = (byte) (listSize);
+			// 5/ update the list byte size
+			int listEnd = byteStream.getCount();
+			int listSize = listEnd - listStart;
+			byte[] bytes = byteStream.getData();
+			bytes[byteSizeStart] = (byte) (listSize >> 24);
+			bytes[byteSizeStart + 1] = (byte) (listSize >> 16);
+			bytes[byteSizeStart + 2] = (byte) (listSize >> 8);
+			bytes[byteSizeStart + 3] = (byte) (listSize);
 
-                return;
-            }
-            case MAP: {
-                MapObjectInspector moi = (MapObjectInspector) objInspector;
-                ObjectInspector koi = moi.getMapKeyObjectInspector();
-                ObjectInspector voi = moi.getMapValueObjectInspector();
-                Map<?, ?> map = moi.getMap(obj);
+			return;
+		}
+		case MAP: {
+			MapObjectInspector moi = (MapObjectInspector) objInspector;
+			ObjectInspector koi = moi.getMapKeyObjectInspector();
+			ObjectInspector voi = moi.getMapValueObjectInspector();
+			Map<?, ?> map = moi.getMap(obj);
 
-                // 1/ reserve spaces for the byte size of the map
-                // which is a integer and takes four bytes
-                int byteSizeStart = byteStream.getCount();
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                int mapStart = byteStream.getCount();
+			// 1/ reserve spaces for the byte size of the map
+			// which is a integer and takes four bytes
+			int byteSizeStart = byteStream.getCount();
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			int mapStart = byteStream.getCount();
 
-                // 2/ write the size of the map which is a VInt
-                int size = map.size();
-                LazyUtils.writeVInt(byteStream, size);
+			// 2/ write the size of the map which is a VInt
+			int size = map.size();
+			LazyUtils.writeVInt(byteStream, size);
 
-                // 3/ write the null bytes
-                int b = 0;
-                byte nullByte = 0;
-                for (Map.Entry<?, ?> entry : map.entrySet()) {
-                    // set the bit to 1 if a key is not null
-                    if (null != entry.getKey()) {
-                        nullByte |= 1 << (b % 8);
-                    } else if (!nullMapKey) {
-                        nullMapKey = true;
-                        LOG.warn("Null map key encountered! Ignoring similar problems.");
-                    }
-                    b++;
-                    // set the bit to 1 if a value is not null
-                    if (null != entry.getValue()) {
-                        nullByte |= 1 << (b % 8);
-                    }
-                    b++;
-                    // write the byte to stream every 4 key-value pairs
-                    // or if this is the last key-value pair
-                    if (0 == b % 8 || b == size * 2) {
-                        byteStream.write(nullByte);
-                        nullByte = 0;
-                    }
-                }
+			// 3/ write the null bytes
+			int b = 0;
+			byte nullByte = 0;
+			for (Map.Entry<?, ?> entry : map.entrySet()) {
+				// set the bit to 1 if a key is not null
+				if (null != entry.getKey()) {
+					nullByte |= 1 << (b % 8);
+				} else if (!nullMapKey) {
+					nullMapKey = true;
+					LOG.warn("Null map key encountered! Ignoring similar problems.");
+				}
+				b++;
+				// set the bit to 1 if a value is not null
+				if (null != entry.getValue()) {
+					nullByte |= 1 << (b % 8);
+				}
+				b++;
+				// write the byte to stream every 4 key-value pairs
+				// or if this is the last key-value pair
+				if (0 == b % 8 || b == size * 2) {
+					byteStream.write(nullByte);
+					nullByte = 0;
+				}
+			}
 
-                // 4/ write key-value pairs one by one
-                for (Map.Entry<?, ?> entry : map.entrySet()) {
-                    serialize(byteStream, entry.getKey(), koi);
-                    serialize(byteStream, entry.getValue(), voi);
-                }
+			// 4/ write key-value pairs one by one
+			for (Map.Entry<?, ?> entry : map.entrySet()) {
+				serialize(byteStream, entry.getKey(), koi);
+				serialize(byteStream, entry.getValue(), voi);
+			}
 
-                // 5/ update the byte size of the map
-                int mapEnd = byteStream.getCount();
-                int mapSize = mapEnd - mapStart;
-                byte[] bytes = byteStream.getData();
-                bytes[byteSizeStart] = (byte) (mapSize >> 24);
-                bytes[byteSizeStart + 1] = (byte) (mapSize >> 16);
-                bytes[byteSizeStart + 2] = (byte) (mapSize >> 8);
-                bytes[byteSizeStart + 3] = (byte) (mapSize);
+			// 5/ update the byte size of the map
+			int mapEnd = byteStream.getCount();
+			int mapSize = mapEnd - mapStart;
+			byte[] bytes = byteStream.getData();
+			bytes[byteSizeStart] = (byte) (mapSize >> 24);
+			bytes[byteSizeStart + 1] = (byte) (mapSize >> 16);
+			bytes[byteSizeStart + 2] = (byte) (mapSize >> 8);
+			bytes[byteSizeStart + 3] = (byte) (mapSize);
 
-                return;
-            }
-            case STRUCT: {
-                // 1/ reserve spaces for the byte size of the struct
-                // which is a integer and takes four bytes
-                int byteSizeStart = byteStream.getCount();
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                byteStream.write((byte) 0);
-                int structStart = byteStream.getCount();
+			return;
+		}
+		case STRUCT: {
+			// 1/ reserve spaces for the byte size of the struct
+			// which is a integer and takes four bytes
+			int byteSizeStart = byteStream.getCount();
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			byteStream.write((byte) 0);
+			int structStart = byteStream.getCount();
 
-                // 2/ serialize the struct
-                serializeStruct(byteStream, obj, (StructObjectInspector) objInspector);
+			// 2/ serialize the struct
+			serializeStruct(byteStream, obj,
+					(StructObjectInspector) objInspector);
 
-                // 3/ update the byte size of the struct
-                int structEnd = byteStream.getCount();
-                int structSize = structEnd - structStart;
-                byte[] bytes = byteStream.getData();
-                bytes[byteSizeStart] = (byte) (structSize >> 24);
-                bytes[byteSizeStart + 1] = (byte) (structSize >> 16);
-                bytes[byteSizeStart + 2] = (byte) (structSize >> 8);
-                bytes[byteSizeStart + 3] = (byte) (structSize);
+			// 3/ update the byte size of the struct
+			int structEnd = byteStream.getCount();
+			int structSize = structEnd - structStart;
+			byte[] bytes = byteStream.getData();
+			bytes[byteSizeStart] = (byte) (structSize >> 24);
+			bytes[byteSizeStart + 1] = (byte) (structSize >> 16);
+			bytes[byteSizeStart + 2] = (byte) (structSize >> 8);
+			bytes[byteSizeStart + 3] = (byte) (structSize);
 
-                return;
-            }
-            default: {
-                throw new RuntimeException("Unrecognized type: " + objInspector.getCategory());
-            }
-        }
-    }
+			return;
+		}
+		default: {
+			throw new RuntimeException("Unrecognized type: "
+					+ objInspector.getCategory());
+		}
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java
index 4588c58..7484b72 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyShort.java
@@ -34,28 +34,29 @@
  * </p>
  * 
  */
-public class LazyShort extends LazyPrimitive<LazyShortObjectInspector, ShortWritable> {
+public class LazyShort extends
+		LazyPrimitive<LazyShortObjectInspector, ShortWritable> {
 
-    public LazyShort(LazyShortObjectInspector oi) {
-        super(oi);
-        data = new ShortWritable();
-    }
+	public LazyShort(LazyShortObjectInspector oi) {
+		super(oi);
+		data = new ShortWritable();
+	}
 
-    public LazyShort(LazyShort copy) {
-        super(copy);
-        data = new ShortWritable(copy.data.get());
-    }
+	public LazyShort(LazyShort copy) {
+		super(copy);
+		data = new ShortWritable(copy.data.get());
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
-        
-        assert (2 == length);
-        data.set(LazyUtils.byteArrayToShort(bytes, start));
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
+
+		assert (2 == length);
+		data.set(LazyUtils.byteArrayToShort(bytes, start));
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java
index a11b1e1..c13533b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyString.java
@@ -27,33 +27,34 @@
  */
 public class LazyString extends LazyPrimitive<LazyStringObjectInspector, Text> {
 
-    public LazyString(LazyStringObjectInspector oi) {
-        super(oi);
-        data = new Text();
-    }
+	public LazyString(LazyStringObjectInspector oi) {
+		super(oi);
+		data = new Text();
+	}
 
-    public LazyString(LazyString copy) {
-        super(copy);
-        data = new Text(copy.data);
-    }
+	public LazyString(LazyString copy) {
+		super(copy);
+		data = new Text(copy.data);
+	}
 
-    VInt vInt = new LazyUtils.VInt();
+	VInt vInt = new LazyUtils.VInt();
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        if (length == 0) {
-            isNull = true;
-            return;
-        } else
-            isNull = false;
-        
-        // get the byte length of the string
-        LazyUtils.readVInt(bytes, start, vInt);
-        if (vInt.value + vInt.length != length)
-            throw new IllegalStateException("parse string: length mismatch, expected " + (vInt.value + vInt.length)
-                    + " but get " + length);
-        assert (length - vInt.length > -1);
-        data.set(bytes, start + vInt.length, length - vInt.length);
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		if (length == 0) {
+			isNull = true;
+			return;
+		} else
+			isNull = false;
+
+		// get the byte length of the string
+		LazyUtils.readVInt(bytes, start, vInt);
+		if (vInt.value + vInt.length != length)
+			throw new IllegalStateException(
+					"parse string: length mismatch, expected "
+							+ (vInt.value + vInt.length) + " but get " + length);
+		assert (length - vInt.length > -1);
+		data.set(bytes, start + vInt.length, length - vInt.length);
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java
index 3ab59b3..61cc335 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyStruct.java
@@ -43,194 +43,202 @@
  */
 public class LazyStruct extends LazyNonPrimitive<LazyStructObjectInspector> {
 
-    private static Log LOG = LogFactory.getLog(LazyStruct.class.getName());
+	private static Log LOG = LogFactory.getLog(LazyStruct.class.getName());
 
-    /**
-     * Whether the data is already parsed or not.
-     */
-    boolean parsed;
+	/**
+	 * Whether the data is already parsed or not.
+	 */
+	boolean parsed;
 
-    /**
-     * The fields of the struct.
-     */
-    LazyObject[] fields;
+	/**
+	 * The fields of the struct.
+	 */
+	@SuppressWarnings("rawtypes")
+	LazyObject[] fields;
 
-    /**
-     * Whether a field is initialized or not.
-     */
-    boolean[] fieldInited;
+	/**
+	 * Whether a field is initialized or not.
+	 */
+	boolean[] fieldInited;
 
-    /**
-     * Whether a field is null or not. Because length is 0 does not means the
-     * field is null. In particular, a 0-length string is not null.
-     */
-    boolean[] fieldIsNull;
+	/**
+	 * Whether a field is null or not. Because length is 0 does not means the
+	 * field is null. In particular, a 0-length string is not null.
+	 */
+	boolean[] fieldIsNull;
 
-    /**
-     * The start positions and lengths of struct fields. Only valid when the
-     * data is parsed.
-     */
-    int[] fieldStart;
-    int[] fieldLength;
+	/**
+	 * The start positions and lengths of struct fields. Only valid when the
+	 * data is parsed.
+	 */
+	int[] fieldStart;
+	int[] fieldLength;
 
-    /**
-     * Construct a LazyStruct object with an ObjectInspector.
-     */
-    protected LazyStruct(LazyStructObjectInspector oi) {
-        super(oi);
-    }
+	/**
+	 * Construct a LazyStruct object with an ObjectInspector.
+	 */
+	protected LazyStruct(LazyStructObjectInspector oi) {
+		super(oi);
+	}
 
-    @Override
-    public void init(byte[] bytes, int start, int length) {
-        super.init(bytes, start, length);
-        parsed = false;
-    }
+	@Override
+	public void init(byte[] bytes, int start, int length) {
+		super.init(bytes, start, length);
+		parsed = false;
+	}
 
-    RecordInfo recordInfo = new LazyUtils.RecordInfo();
-    boolean missingFieldWarned = false;
-    boolean extraFieldWarned = false;
+	RecordInfo recordInfo = new LazyUtils.RecordInfo();
+	boolean missingFieldWarned = false;
+	boolean extraFieldWarned = false;
 
-    /**
-     * Parse the byte[] and fill fieldStart, fieldLength, fieldInited and
-     * fieldIsNull.
-     */
-    private void parse() {
+	/**
+	 * Parse the byte[] and fill fieldStart, fieldLength, fieldInited and
+	 * fieldIsNull.
+	 */
+	private void parse() {
 
-        List<? extends StructField> fieldRefs = ((StructObjectInspector) oi).getAllStructFieldRefs();
+		List<? extends StructField> fieldRefs = ((StructObjectInspector) oi)
+				.getAllStructFieldRefs();
 
-        if (fields == null) {
-            fields = new LazyObject[fieldRefs.size()];
-            for (int i = 0; i < fields.length; i++) {
-                ObjectInspector insp = fieldRefs.get(i).getFieldObjectInspector();
-                fields[i] = insp == null ? null : LazyFactory.createLazyObject(insp);
-            }
-            fieldInited = new boolean[fields.length];
-            fieldIsNull = new boolean[fields.length];
-            fieldStart = new int[fields.length];
-            fieldLength = new int[fields.length];
-        }
+		if (fields == null) {
+			fields = new LazyObject[fieldRefs.size()];
+			for (int i = 0; i < fields.length; i++) {
+				ObjectInspector insp = fieldRefs.get(i)
+						.getFieldObjectInspector();
+				fields[i] = insp == null ? null : LazyFactory
+						.createLazyObject(insp);
+			}
+			fieldInited = new boolean[fields.length];
+			fieldIsNull = new boolean[fields.length];
+			fieldStart = new int[fields.length];
+			fieldLength = new int[fields.length];
+		}
 
-        /**
-         * Please note that one null byte is followed by eight fields, then more
-         * null byte and fields.
-         */
+		/**
+		 * Please note that one null byte is followed by eight fields, then more
+		 * null byte and fields.
+		 */
 
-        int fieldId = 0;
-        int structByteEnd = start + length;
+		int fieldId = 0;
+		int structByteEnd = start + length;
 
-        byte nullByte = bytes[start];
-        int lastFieldByteEnd = start + 1;
-        // Go through all bytes in the byte[]
-        for (int i = 0; i < fields.length; i++) {
-            fieldIsNull[i] = true;
-            if ((nullByte & (1 << (i % 8))) != 0) {
-                fieldIsNull[i] = false;
-                LazyUtils.checkObjectByteInfo(fieldRefs.get(i).getFieldObjectInspector(), bytes, lastFieldByteEnd,
-                        recordInfo);
-                fieldStart[i] = lastFieldByteEnd + recordInfo.elementOffset;
-                fieldLength[i] = recordInfo.elementSize;
-                lastFieldByteEnd = fieldStart[i] + fieldLength[i];
-            }
+		byte nullByte = bytes[start];
+		int lastFieldByteEnd = start + 1;
+		// Go through all bytes in the byte[]
+		for (int i = 0; i < fields.length; i++) {
+			fieldIsNull[i] = true;
+			if ((nullByte & (1 << (i % 8))) != 0) {
+				fieldIsNull[i] = false;
+				LazyUtils.checkObjectByteInfo(fieldRefs.get(i)
+						.getFieldObjectInspector(), bytes, lastFieldByteEnd,
+						recordInfo);
+				fieldStart[i] = lastFieldByteEnd + recordInfo.elementOffset;
+				fieldLength[i] = recordInfo.elementSize;
+				lastFieldByteEnd = fieldStart[i] + fieldLength[i];
+			}
 
-            // count how many fields are there
-            if (lastFieldByteEnd <= structByteEnd) {
-                fieldId++;
-            }
-            // next byte is a null byte if there are more bytes to go
-            if (7 == (i % 8)) {
-                if (lastFieldByteEnd < structByteEnd) {
-                    nullByte = bytes[lastFieldByteEnd];
-                    lastFieldByteEnd++;
-                } else {
-                    // otherwise all null afterwards
-                    nullByte = 0;
-                    lastFieldByteEnd++;
-                }
-            }
-        }
+			// count how many fields are there
+			if (lastFieldByteEnd <= structByteEnd) {
+				fieldId++;
+			}
+			// next byte is a null byte if there are more bytes to go
+			if (7 == (i % 8)) {
+				if (lastFieldByteEnd < structByteEnd) {
+					nullByte = bytes[lastFieldByteEnd];
+					lastFieldByteEnd++;
+				} else {
+					// otherwise all null afterwards
+					nullByte = 0;
+					lastFieldByteEnd++;
+				}
+			}
+		}
 
-        // Extra bytes at the end?
-        if (!extraFieldWarned && lastFieldByteEnd < structByteEnd) {
-            extraFieldWarned = true;
-            LOG.warn("Extra bytes detected at the end of the row! Ignoring similar " + "problems.");
-        }
+		// Extra bytes at the end?
+		if (!extraFieldWarned && lastFieldByteEnd < structByteEnd) {
+			extraFieldWarned = true;
+			LOG.warn("Extra bytes detected at the end of the row! Ignoring similar "
+					+ "problems.");
+		}
 
-        // Missing fields?
-        if (!missingFieldWarned && lastFieldByteEnd > structByteEnd) {
-            missingFieldWarned = true;
-            LOG.warn("Missing fields! Expected " + fields.length + " fields but " + "only got " + fieldId
-                    + "! Ignoring similar problems.");
-        }
+		// Missing fields?
+		if (!missingFieldWarned && lastFieldByteEnd > structByteEnd) {
+			missingFieldWarned = true;
+			LOG.warn("Missing fields! Expected " + fields.length
+					+ " fields but " + "only got " + fieldId
+					+ "! Ignoring similar problems.");
+		}
 
-        Arrays.fill(fieldInited, false);
-        parsed = true;
-    }
+		Arrays.fill(fieldInited, false);
+		parsed = true;
+	}
 
-    /**
-     * Get one field out of the struct.
-     * 
-     * If the field is a primitive field, return the actual object. Otherwise
-     * return the LazyObject. This is because PrimitiveObjectInspector does not
-     * have control over the object used by the user - the user simply directly
-     * use the Object instead of going through Object
-     * PrimitiveObjectInspector.get(Object).
-     * 
-     * @param fieldID
-     *            The field ID
-     * @return The field as a LazyObject
-     */
-    public Object getField(int fieldID) {
-        if (!parsed) {
-            parse();
-        }
-        return uncheckedGetField(fieldID);
-    }
+	/**
+	 * Get one field out of the struct.
+	 * 
+	 * If the field is a primitive field, return the actual object. Otherwise
+	 * return the LazyObject. This is because PrimitiveObjectInspector does not
+	 * have control over the object used by the user - the user simply directly
+	 * use the Object instead of going through Object
+	 * PrimitiveObjectInspector.get(Object).
+	 * 
+	 * @param fieldID
+	 *            The field ID
+	 * @return The field as a LazyObject
+	 */
+	public Object getField(int fieldID) {
+		if (!parsed) {
+			parse();
+		}
+		return uncheckedGetField(fieldID);
+	}
 
-    /**
-     * Get the field out of the row without checking parsed. This is called by
-     * both getField and getFieldsAsList.
-     * 
-     * @param fieldID
-     *            The id of the field starting from 0.
-     * @return The value of the field
-     */
-    private Object uncheckedGetField(int fieldID) {
-        // Test the length first so in most cases we avoid doing a byte[]
-        // comparison.
-        if (fieldIsNull[fieldID]) {
-            return null;
-        }
-        if (!fieldInited[fieldID]) {
-            fieldInited[fieldID] = true;
-            fields[fieldID].init(bytes, fieldStart[fieldID], fieldLength[fieldID]);
-        }
-        return fields[fieldID].getObject();
-    }
+	/**
+	 * Get the field out of the row without checking parsed. This is called by
+	 * both getField and getFieldsAsList.
+	 * 
+	 * @param fieldID
+	 *            The id of the field starting from 0.
+	 * @return The value of the field
+	 */
+	private Object uncheckedGetField(int fieldID) {
+		// Test the length first so in most cases we avoid doing a byte[]
+		// comparison.
+		if (fieldIsNull[fieldID]) {
+			return null;
+		}
+		if (!fieldInited[fieldID]) {
+			fieldInited[fieldID] = true;
+			fields[fieldID].init(bytes, fieldStart[fieldID],
+					fieldLength[fieldID]);
+		}
+		return fields[fieldID].getObject();
+	}
 
-    ArrayList<Object> cachedList;
+	ArrayList<Object> cachedList;
 
-    /**
-     * Get the values of the fields as an ArrayList.
-     * 
-     * @return The values of the fields as an ArrayList.
-     */
-    public ArrayList<Object> getFieldsAsList() {
-        if (!parsed) {
-            parse();
-        }
-        if (cachedList == null) {
-            cachedList = new ArrayList<Object>();
-        } else {
-            cachedList.clear();
-        }
-        for (int i = 0; i < fields.length; i++) {
-            cachedList.add(uncheckedGetField(i));
-        }
-        return cachedList;
-    }
+	/**
+	 * Get the values of the fields as an ArrayList.
+	 * 
+	 * @return The values of the fields as an ArrayList.
+	 */
+	public ArrayList<Object> getFieldsAsList() {
+		if (!parsed) {
+			parse();
+		}
+		if (cachedList == null) {
+			cachedList = new ArrayList<Object>();
+		} else {
+			cachedList.clear();
+		}
+		for (int i = 0; i < fields.length; i++) {
+			cachedList.add(uncheckedGetField(i));
+		}
+		return cachedList;
+	}
 
-    @Override
-    public Object getObject() {
-        return this;
-    }
+	@Override
+	public Object getObject() {
+		return this;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java
index d7baf18..2d0406c 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/LazyUtils.java
@@ -25,8 +25,6 @@
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
@@ -48,462 +46,484 @@
  */
 public final class LazyUtils {
 
-    private static Log LOG = LogFactory.getLog(LazyUtils.class.getName());
+	/**
+	 * Convert the byte array to an int starting from the given offset. Refer to
+	 * code by aeden on DZone Snippets:
+	 * 
+	 * @param b
+	 *            the byte array
+	 * @param offset
+	 *            the array offset
+	 * @return the integer
+	 */
+	public static int byteArrayToInt(byte[] b, int offset) {
+		int value = 0;
+		for (int i = 0; i < 4; i++) {
+			int shift = (4 - 1 - i) * 8;
+			value += (b[i + offset] & 0x000000FF) << shift;
+		}
+		return value;
+	}
 
-    /**
-     * Convert the byte array to an int starting from the given offset. Refer to
-     * code by aeden on DZone Snippets:
-     * 
-     * @param b
-     *            the byte array
-     * @param offset
-     *            the array offset
-     * @return the integer
-     */
-    public static int byteArrayToInt(byte[] b, int offset) {
-        int value = 0;
-        for (int i = 0; i < 4; i++) {
-            int shift = (4 - 1 - i) * 8;
-            value += (b[i + offset] & 0x000000FF) << shift;
-        }
-        return value;
-    }
+	/**
+	 * Convert the byte array to a long starting from the given offset.
+	 * 
+	 * @param b
+	 *            the byte array
+	 * @param offset
+	 *            the array offset
+	 * @return the long
+	 */
+	public static long byteArrayToLong(byte[] b, int offset) {
+		long value = 0;
+		for (int i = 0; i < 8; i++) {
+			int shift = (8 - 1 - i) * 8;
+			value += ((long) (b[i + offset] & 0x00000000000000FF)) << shift;
+		}
+		return value;
+	}
 
-    /**
-     * Convert the byte array to a long starting from the given offset.
-     * 
-     * @param b
-     *            the byte array
-     * @param offset
-     *            the array offset
-     * @return the long
-     */
-    public static long byteArrayToLong(byte[] b, int offset) {
-        long value = 0;
-        for (int i = 0; i < 8; i++) {
-            int shift = (8 - 1 - i) * 8;
-            value += ((long) (b[i + offset] & 0x00000000000000FF)) << shift;
-        }
-        return value;
-    }
+	/**
+	 * Convert the byte array to a short starting from the given offset.
+	 * 
+	 * @param b
+	 *            the byte array
+	 * @param offset
+	 *            the array offset
+	 * @return the short
+	 */
+	public static short byteArrayToShort(byte[] b, int offset) {
+		short value = 0;
+		value += (b[offset] & 0x000000FF) << 8;
+		value += (b[offset + 1] & 0x000000FF);
+		return value;
+	}
 
-    /**
-     * Convert the byte array to a short starting from the given offset.
-     * 
-     * @param b
-     *            the byte array
-     * @param offset
-     *            the array offset
-     * @return the short
-     */
-    public static short byteArrayToShort(byte[] b, int offset) {
-        short value = 0;
-        value += (b[offset] & 0x000000FF) << 8;
-        value += (b[offset + 1] & 0x000000FF);
-        return value;
-    }
+	/**
+	 * Record is the unit that data is serialized in. A record includes two
+	 * parts. The first part stores the size of the element and the second part
+	 * stores the real element. size element record ->
+	 * |----|-------------------------|
+	 * 
+	 * A RecordInfo stores two information of a record, the size of the "size"
+	 * part which is the element offset and the size of the element part which
+	 * is element size.
+	 */
+	public static class RecordInfo {
+		public RecordInfo() {
+			elementOffset = 0;
+			elementSize = 0;
+		}
 
-    /**
-     * Record is the unit that data is serialized in. A record includes two
-     * parts. The first part stores the size of the element and the second part
-     * stores the real element. size element record ->
-     * |----|-------------------------|
-     * 
-     * A RecordInfo stores two information of a record, the size of the "size"
-     * part which is the element offset and the size of the element part which
-     * is element size.
-     */
-    public static class RecordInfo {
-        public RecordInfo() {
-            elementOffset = 0;
-            elementSize = 0;
-        }
+		public byte elementOffset;
+		public int elementSize;
 
-        public byte elementOffset;
-        public int elementSize;
+		@Override
+		public String toString() {
+			return "(" + elementOffset + ", " + elementSize + ")";
+		}
+	}
 
-        @Override
-        public String toString() {
-            return "(" + elementOffset + ", " + elementSize + ")";
-        }
-    }
+	static VInt vInt = new LazyUtils.VInt();
 
-    static VInt vInt = new LazyUtils.VInt();
+	/**
+	 * Check a particular field and set its size and offset in bytes based on
+	 * the field type and the bytes arrays.
+	 * 
+	 * For void, boolean, byte, short, int, long, float and double, there is no
+	 * offset and the size is fixed. For string, map, list, struct, the first
+	 * four bytes are used to store the size. So the offset is 4 and the size is
+	 * computed by concating the first four bytes together. The first four bytes
+	 * are defined with respect to the offset in the bytes arrays.
+	 * 
+	 * @param objectInspector
+	 *            object inspector of the field
+	 * @param bytes
+	 *            bytes arrays store the table row
+	 * @param offset
+	 *            offset of this field
+	 * @param recordInfo
+	 *            modify this byteinfo object and return it
+	 */
+	public static void checkObjectByteInfo(ObjectInspector objectInspector,
+			byte[] bytes, int offset, RecordInfo recordInfo) {
+		Category category = objectInspector.getCategory();
+		switch (category) {
+		case PRIMITIVE:
+			PrimitiveCategory primitiveCategory = ((PrimitiveObjectInspector) objectInspector)
+					.getPrimitiveCategory();
+			switch (primitiveCategory) {
+			case VOID:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = 0;
+				break;
+			case BOOLEAN:
+			case BYTE:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = 1;
+				break;
+			case SHORT:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = 2;
+				break;
+			case FLOAT:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = 4;
+				break;
+			case DOUBLE:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = 8;
+				break;
+			case INT:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = WritableUtils
+						.decodeVIntSize(bytes[offset]);
+				break;
+			case LONG:
+				recordInfo.elementOffset = 0;
+				recordInfo.elementSize = WritableUtils
+						.decodeVIntSize(bytes[offset]);
+				break;
+			case STRING:
+				// using vint instead of 4 bytes
+				LazyUtils.readVInt(bytes, offset, vInt);
+				recordInfo.elementOffset = vInt.length;
+				recordInfo.elementSize = vInt.value;
+				break;
+			default: {
+				throw new RuntimeException("Unrecognized primitive type: "
+						+ primitiveCategory);
+			}
+			}
+			break;
+		case LIST:
+		case MAP:
+		case STRUCT:
+			recordInfo.elementOffset = 4;
+			recordInfo.elementSize = LazyUtils.byteArrayToInt(bytes, offset);
+			break;
+		default: {
+			throw new RuntimeException("Unrecognized non-primitive type: "
+					+ category);
+		}
+		}
+	}
 
-    /**
-     * Check a particular field and set its size and offset in bytes based on
-     * the field type and the bytes arrays.
-     * 
-     * For void, boolean, byte, short, int, long, float and double, there is no
-     * offset and the size is fixed. For string, map, list, struct, the first
-     * four bytes are used to store the size. So the offset is 4 and the size is
-     * computed by concating the first four bytes together. The first four bytes
-     * are defined with respect to the offset in the bytes arrays.
-     * 
-     * @param objectInspector
-     *            object inspector of the field
-     * @param bytes
-     *            bytes arrays store the table row
-     * @param offset
-     *            offset of this field
-     * @param recordInfo
-     *            modify this byteinfo object and return it
-     */
-    public static void checkObjectByteInfo(ObjectInspector objectInspector, byte[] bytes, int offset,
-            RecordInfo recordInfo) {
-        Category category = objectInspector.getCategory();
-        switch (category) {
-            case PRIMITIVE:
-                PrimitiveCategory primitiveCategory = ((PrimitiveObjectInspector) objectInspector)
-                        .getPrimitiveCategory();
-                switch (primitiveCategory) {
-                    case VOID:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = 0;
-                        break;
-                    case BOOLEAN:
-                    case BYTE:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = 1;
-                        break;
-                    case SHORT:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = 2;
-                        break;
-                    case FLOAT:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = 4;
-                        break;
-                    case DOUBLE:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = 8;
-                        break;
-                    case INT:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
-                        break;
-                    case LONG:
-                        recordInfo.elementOffset = 0;
-                        recordInfo.elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
-                        break;
-                    case STRING:
-                        // using vint instead of 4 bytes
-                        LazyUtils.readVInt(bytes, offset, vInt);
-                        recordInfo.elementOffset = vInt.length;
-                        recordInfo.elementSize = vInt.value;
-                        break;
-                    default: {
-                        throw new RuntimeException("Unrecognized primitive type: " + primitiveCategory);
-                    }
-                }
-                break;
-            case LIST:
-            case MAP:
-            case STRUCT:
-                recordInfo.elementOffset = 4;
-                recordInfo.elementSize = LazyUtils.byteArrayToInt(bytes, offset);
-                break;
-            default: {
-                throw new RuntimeException("Unrecognized non-primitive type: " + category);
-            }
-        }
-    }
+	/**
+	 * A zero-compressed encoded long.
+	 */
+	public static class VLong {
+		public VLong() {
+			value = 0;
+			length = 0;
+		}
 
-    /**
-     * A zero-compressed encoded long.
-     */
-    public static class VLong {
-        public VLong() {
-            value = 0;
-            length = 0;
-        }
+		public long value;
+		public byte length;
+	};
 
-        public long value;
-        public byte length;
-    };
+	/**
+	 * Reads a zero-compressed encoded long from a byte array and returns it.
+	 * 
+	 * @param bytes
+	 *            the byte array
+	 * @param offset
+	 *            offset of the array to read from
+	 * @param vlong
+	 *            storing the deserialized long and its size in byte
+	 */
+	public static void readVLong(byte[] bytes, int offset, VLong vlong) {
+		byte firstByte = bytes[offset];
+		vlong.length = (byte) WritableUtils.decodeVIntSize(firstByte);
+		if (vlong.length == 1) {
+			vlong.value = firstByte;
+			return;
+		}
+		long i = 0;
+		for (int idx = 0; idx < vlong.length - 1; idx++) {
+			byte b = bytes[offset + 1 + idx];
+			i = i << 8;
+			i = i | (b & 0xFF);
+		}
+		vlong.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+	}
 
-    /**
-     * Reads a zero-compressed encoded long from a byte array and returns it.
-     * 
-     * @param bytes
-     *            the byte array
-     * @param offset
-     *            offset of the array to read from
-     * @param vlong
-     *            storing the deserialized long and its size in byte
-     */
-    public static void readVLong(byte[] bytes, int offset, VLong vlong) {
-        byte firstByte = bytes[offset];
-        vlong.length = (byte) WritableUtils.decodeVIntSize(firstByte);
-        if (vlong.length == 1) {
-            vlong.value = firstByte;
-            return;
-        }
-        long i = 0;
-        for (int idx = 0; idx < vlong.length - 1; idx++) {
-            byte b = bytes[offset + 1 + idx];
-            i = i << 8;
-            i = i | (b & 0xFF);
-        }
-        vlong.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
-    }
+	/**
+	 * A zero-compressed encoded integer.
+	 */
+	public static class VInt implements Serializable {
+		private static final long serialVersionUID = 1L;
 
-    /**
-     * A zero-compressed encoded integer.
-     */
-    public static class VInt implements Serializable {
-        public VInt() {
-            value = 0;
-            length = 0;
-        }
+		public VInt() {
+			value = 0;
+			length = 0;
+		}
 
-        public int value;
-        public byte length;
-    };
+		public int value;
+		public byte length;
+	};
 
-    /**
-     * Reads a zero-compressed encoded int from a byte array and returns it.
-     * 
-     * @param bytes
-     *            the byte array
-     * @param offset
-     *            offset of the array to read from
-     * @param vInt
-     *            storing the deserialized int and its size in byte
-     */
-    public static void readVInt(byte[] bytes, int offset, VInt vInt) {
-        byte firstByte = bytes[offset];
-        vInt.length = (byte) WritableUtils.decodeVIntSize(firstByte);
-        if (vInt.length == 1) {
-            vInt.value = firstByte;
-            return;
-        }
-        int i = 0;
-        for (int idx = 0; idx < vInt.length - 1; idx++) {
-            byte b = bytes[offset + 1 + idx];
-            i = i << 8;
-            i = i | (b & 0xFF);
-        }
-        vInt.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1) : i);
-    }
+	/**
+	 * Reads a zero-compressed encoded int from a byte array and returns it.
+	 * 
+	 * @param bytes
+	 *            the byte array
+	 * @param offset
+	 *            offset of the array to read from
+	 * @param vInt
+	 *            storing the deserialized int and its size in byte
+	 */
+	public static void readVInt(byte[] bytes, int offset, VInt vInt) {
+		byte firstByte = bytes[offset];
+		vInt.length = (byte) WritableUtils.decodeVIntSize(firstByte);
+		if (vInt.length == 1) {
+			vInt.value = firstByte;
+			return;
+		}
+		int i = 0;
+		for (int idx = 0; idx < vInt.length - 1; idx++) {
+			byte b = bytes[offset + 1 + idx];
+			i = i << 8;
+			i = i | (b & 0xFF);
+		}
+		vInt.value = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1) : i);
+	}
 
-    /**
-     * Writes a zero-compressed encoded int to a byte array.
-     * 
-     * @param byteStream
-     *            the byte array/stream
-     * @param i
-     *            the int
-     */
-    public static void writeVInt(Output byteStream, int i) {
-        writeVLong(byteStream, i);
-    }
+	/**
+	 * Writes a zero-compressed encoded int to a byte array.
+	 * 
+	 * @param byteStream
+	 *            the byte array/stream
+	 * @param i
+	 *            the int
+	 */
+	public static void writeVInt(Output byteStream, int i) {
+		writeVLong(byteStream, i);
+	}
 
-    /**
-     * Write a zero-compressed encoded long to a byte array.
-     * 
-     * @param byteStream
-     *            the byte array/stream
-     * @param l
-     *            the long
-     */
-    public static void writeVLong(Output byteStream, long l) {
-        if (l >= -112 && l <= 127) {
-            byteStream.write((byte) l);
-            return;
-        }
+	/**
+	 * Write a zero-compressed encoded long to a byte array.
+	 * 
+	 * @param byteStream
+	 *            the byte array/stream
+	 * @param l
+	 *            the long
+	 */
+	public static void writeVLong(Output byteStream, long l) {
+		if (l >= -112 && l <= 127) {
+			byteStream.write((byte) l);
+			return;
+		}
 
-        int len = -112;
-        if (l < 0) {
-            l ^= -1L; // take one's complement'
-            len = -120;
-        }
+		int len = -112;
+		if (l < 0) {
+			l ^= -1L; // take one's complement'
+			len = -120;
+		}
 
-        long tmp = l;
-        while (tmp != 0) {
-            tmp = tmp >> 8;
-            len--;
-        }
+		long tmp = l;
+		while (tmp != 0) {
+			tmp = tmp >> 8;
+			len--;
+		}
 
-        byteStream.write((byte) len);
+		byteStream.write((byte) len);
 
-        len = (len < -120) ? -(len + 120) : -(len + 112);
+		len = (len < -120) ? -(len + 120) : -(len + 112);
 
-        for (int idx = len; idx != 0; idx--) {
-            int shiftbits = (idx - 1) * 8;
-            long mask = 0xFFL << shiftbits;
-            byteStream.write((byte) ((l & mask) >> shiftbits));
-        }
-    }
+		for (int idx = len; idx != 0; idx--) {
+			int shiftbits = (idx - 1) * 8;
+			long mask = 0xFFL << shiftbits;
+			byteStream.write((byte) ((l & mask) >> shiftbits));
+		}
+	}
 
-    static Map<TypeInfo, ObjectInspector> cachedLazyObjectInspector = new ConcurrentHashMap<TypeInfo, ObjectInspector>();
+	static Map<TypeInfo, ObjectInspector> cachedLazyObjectInspector = new ConcurrentHashMap<TypeInfo, ObjectInspector>();
 
-    /**
-     * Returns the lazy binary object inspector that can be used to inspect an
-     * lazy binary object of that typeInfo
-     * 
-     * For primitive types, we use the standard writable object inspector.
-     */
-    public static ObjectInspector getLazyObjectInspectorFromTypeInfo(TypeInfo typeInfo, boolean topLevel) {
-        if (typeInfo == null)
-            throw new IllegalStateException("illegal type null ");
-        ObjectInspector result = cachedLazyObjectInspector.get(typeInfo);
-        if (result == null) {
-            switch (typeInfo.getCategory()) {
-                case PRIMITIVE: {
-                    result = PrimitiveObjectInspectorFactory
-                            .getPrimitiveLazyObjectInspector(((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory());
-                    break;
-                }
-                case LIST: {
-                    ObjectInspector elementObjectInspector = getLazyObjectInspectorFromTypeInfo(
-                            ((ListTypeInfo) typeInfo).getListElementTypeInfo(), false);
-                    result = LazyObjectInspectorFactory.getLazyListObjectInspector(elementObjectInspector);
-                    break;
-                }
-                case MAP: {
-                    MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo;
-                    ObjectInspector keyObjectInspector = getLazyObjectInspectorFromTypeInfo(
-                            mapTypeInfo.getMapKeyTypeInfo(), false);
-                    ObjectInspector valueObjectInspector = getLazyObjectInspectorFromTypeInfo(
-                            mapTypeInfo.getMapValueTypeInfo(), false);
-                    result = LazyObjectInspectorFactory.getLazyMapObjectInspector(keyObjectInspector,
-                            valueObjectInspector);
-                    break;
-                }
-                case STRUCT: {
-                    StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
-                    List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
-                    List<TypeInfo> fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos();
-                    List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
+	/**
+	 * Returns the lazy binary object inspector that can be used to inspect an
+	 * lazy binary object of that typeInfo
+	 * 
+	 * For primitive types, we use the standard writable object inspector.
+	 */
+	public static ObjectInspector getLazyObjectInspectorFromTypeInfo(
+			TypeInfo typeInfo, boolean topLevel) {
+		if (typeInfo == null)
+			throw new IllegalStateException("illegal type null ");
+		ObjectInspector result = cachedLazyObjectInspector.get(typeInfo);
+		if (result == null) {
+			switch (typeInfo.getCategory()) {
+			case PRIMITIVE: {
+				result = PrimitiveObjectInspectorFactory
+						.getPrimitiveLazyObjectInspector(((PrimitiveTypeInfo) typeInfo)
+								.getPrimitiveCategory());
+				break;
+			}
+			case LIST: {
+				ObjectInspector elementObjectInspector = getLazyObjectInspectorFromTypeInfo(
+						((ListTypeInfo) typeInfo).getListElementTypeInfo(),
+						false);
+				result = LazyObjectInspectorFactory
+						.getLazyListObjectInspector(elementObjectInspector);
+				break;
+			}
+			case MAP: {
+				MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo;
+				ObjectInspector keyObjectInspector = getLazyObjectInspectorFromTypeInfo(
+						mapTypeInfo.getMapKeyTypeInfo(), false);
+				ObjectInspector valueObjectInspector = getLazyObjectInspectorFromTypeInfo(
+						mapTypeInfo.getMapValueTypeInfo(), false);
+				result = LazyObjectInspectorFactory.getLazyMapObjectInspector(
+						keyObjectInspector, valueObjectInspector);
+				break;
+			}
+			case STRUCT: {
+				StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
+				List<String> fieldNames = structTypeInfo
+						.getAllStructFieldNames();
+				List<TypeInfo> fieldTypeInfos = structTypeInfo
+						.getAllStructFieldTypeInfos();
+				List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(
+						fieldTypeInfos.size());
 
-                    for (int i = 0; i < fieldTypeInfos.size(); i++) {
-                        fieldObjectInspectors.add(getLazyObjectInspectorFromTypeInfo(fieldTypeInfos.get(i), false));
-                    }
+				for (int i = 0; i < fieldTypeInfos.size(); i++) {
+					fieldObjectInspectors
+							.add(getLazyObjectInspectorFromTypeInfo(
+									fieldTypeInfos.get(i), false));
+				}
 
-                    // if it is top level then create columnar
-                    if (topLevel)
-                        result = LazyObjectInspectorFactory.getLazyColumnarObjectInspector(fieldNames,
-                                fieldObjectInspectors);
-                    // if it is not top level then create struct
-                    else
-                        result = LazyObjectInspectorFactory.getLazyStructObjectInspector(fieldNames,
-                                fieldObjectInspectors);
+				// if it is top level then create columnar
+				if (topLevel)
+					result = LazyObjectInspectorFactory
+							.getLazyColumnarObjectInspector(fieldNames,
+									fieldObjectInspectors);
+				// if it is not top level then create struct
+				else
+					result = LazyObjectInspectorFactory
+							.getLazyStructObjectInspector(fieldNames,
+									fieldObjectInspectors);
 
-                    break;
-                }
-                default: {
-                    result = null;
-                }
-            }
-            cachedLazyObjectInspector.put(typeInfo, result);
-        }
-        return result;
-    }
+				break;
+			}
+			default: {
+				result = null;
+			}
+			}
+			cachedLazyObjectInspector.put(typeInfo, result);
+		}
+		return result;
+	}
 
-    /**
-     * get top-level lazy object inspector
-     * 
-     * @param fieldNames
-     * @param fieldTypeInfos
-     * @return
-     */
-    public static ObjectInspector getLazyObjectInspector(List<String> fieldNames, List<TypeInfo> fieldTypeInfos) {
-        List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(fieldTypeInfos.size());
-        for (int i = 0; i < fieldTypeInfos.size(); i++) {
-            fieldObjectInspectors.add(getLazyObjectInspectorFromTypeInfo(fieldTypeInfos.get(i), false));
-        }
+	/**
+	 * get top-level lazy object inspector
+	 * 
+	 * @param fieldNames
+	 * @param fieldTypeInfos
+	 * @return
+	 */
+	public static ObjectInspector getLazyObjectInspector(
+			List<String> fieldNames, List<TypeInfo> fieldTypeInfos) {
+		List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>(
+				fieldTypeInfos.size());
+		for (int i = 0; i < fieldTypeInfos.size(); i++) {
+			fieldObjectInspectors.add(getLazyObjectInspectorFromTypeInfo(
+					fieldTypeInfos.get(i), false));
+		}
 
-        return LazyObjectInspectorFactory.getLazyColumnarObjectInspector(fieldNames, fieldObjectInspectors);
-    }
+		return LazyObjectInspectorFactory.getLazyColumnarObjectInspector(
+				fieldNames, fieldObjectInspectors);
+	}
 
-    private LazyUtils() {
-        // prevent instantiation
-    }
+	private LazyUtils() {
+		// prevent instantiation
+	}
 
-    /**
-     * Returns -1 if the first byte sequence is lexicographically less than the
-     * second; returns +1 if the second byte sequence is lexicographically less
-     * than the first; otherwise return 0.
-     */
-    public static int compare(byte[] b1, int start1, int length1, byte[] b2, int start2, int length2) {
+	/**
+	 * Returns -1 if the first byte sequence is lexicographically less than the
+	 * second; returns +1 if the second byte sequence is lexicographically less
+	 * than the first; otherwise return 0.
+	 */
+	public static int compare(byte[] b1, int start1, int length1, byte[] b2,
+			int start2, int length2) {
 
-        int min = Math.min(length1, length2);
+		int min = Math.min(length1, length2);
 
-        for (int i = 0; i < min; i++) {
-            if (b1[start1 + i] == b2[start2 + i]) {
-                continue;
-            }
-            if (b1[start1 + i] < b2[start2 + i]) {
-                return -1;
-            } else {
-                return 1;
-            }
-        }
+		for (int i = 0; i < min; i++) {
+			if (b1[start1 + i] == b2[start2 + i]) {
+				continue;
+			}
+			if (b1[start1 + i] < b2[start2 + i]) {
+				return -1;
+			} else {
+				return 1;
+			}
+		}
 
-        if (length1 < length2) {
-            return -1;
-        }
-        if (length1 > length2) {
-            return 1;
-        }
-        return 0;
-    }
+		if (length1 < length2) {
+			return -1;
+		}
+		if (length1 > length2) {
+			return 1;
+		}
+		return 0;
+	}
 
-    public static int hashBytes(byte[] data, int start, int len) {
-        int hash = 1;
-        for (int i = start; i < len; i++) {
-            hash = (31 * hash) + data[i];
-        }
-        return hash;
-    }
+	public static int hashBytes(byte[] data, int start, int len) {
+		int hash = 1;
+		for (int i = start; i < len; i++) {
+			hash = (31 * hash) + data[i];
+		}
+		return hash;
+	}
 
-    /**
-     * Writes a zero-compressed encoded int to a byte array.
-     * 
-     * @param byteStream
-     *            the byte array/stream
-     * @param i
-     *            the int
-     */
-    public static void writeVInt(DataOutput byteStream, int i) throws IOException {
-        writeVLong(byteStream, i);
-    }
+	/**
+	 * Writes a zero-compressed encoded int to a byte array.
+	 * 
+	 * @param byteStream
+	 *            the byte array/stream
+	 * @param i
+	 *            the int
+	 */
+	public static void writeVInt(DataOutput byteStream, int i)
+			throws IOException {
+		writeVLong(byteStream, i);
+	}
 
-    /**
-     * Write a zero-compressed encoded long to a byte array.
-     * 
-     * @param byteStream
-     *            the byte array/stream
-     * @param l
-     *            the long
-     */
-    public static void writeVLong(DataOutput byteStream, long l) throws IOException {
-        if (l >= -112 && l <= 127) {
-            byteStream.write((byte) l);
-            return;
-        }
+	/**
+	 * Write a zero-compressed encoded long to a byte array.
+	 * 
+	 * @param byteStream
+	 *            the byte array/stream
+	 * @param l
+	 *            the long
+	 */
+	public static void writeVLong(DataOutput byteStream, long l)
+			throws IOException {
+		if (l >= -112 && l <= 127) {
+			byteStream.write((byte) l);
+			return;
+		}
 
-        int len = -112;
-        if (l < 0) {
-            l ^= -1L; // take one's complement'
-            len = -120;
-        }
+		int len = -112;
+		if (l < 0) {
+			l ^= -1L; // take one's complement'
+			len = -120;
+		}
 
-        long tmp = l;
-        while (tmp != 0) {
-            tmp = tmp >> 8;
-            len--;
-        }
+		long tmp = l;
+		while (tmp != 0) {
+			tmp = tmp >> 8;
+			len--;
+		}
 
-        byteStream.write((byte) len);
+		byteStream.write((byte) len);
 
-        len = (len < -120) ? -(len + 120) : -(len + 112);
+		len = (len < -120) ? -(len + 120) : -(len + 112);
 
-        for (int idx = len; idx != 0; idx--) {
-            int shiftbits = (idx - 1) * 8;
-            long mask = 0xFFL << shiftbits;
-            byteStream.write((byte) ((l & mask) >> shiftbits));
-        }
-    }
+		for (int idx = len; idx != 0; idx--) {
+			int shiftbits = (idx - 1) * 8;
+			long mask = 0xFFL << shiftbits;
+			byteStream.write((byte) ((l & mask) >> shiftbits));
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java
index b1ca622..b20f185 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyColumnarObjectInspector.java
@@ -31,47 +31,50 @@
  * 
  * @see LazyColumnar
  */
-public class LazyColumnarObjectInspector extends StandardStructObjectInspector implements Serializable {
+public class LazyColumnarObjectInspector extends StandardStructObjectInspector
+		implements Serializable {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    public LazyColumnarObjectInspector(List<String> structFieldNames, List<ObjectInspector> structFieldObjectInspectors) {
-        super(structFieldNames, structFieldObjectInspectors);
-    }
+	public LazyColumnarObjectInspector(List<String> structFieldNames,
+			List<ObjectInspector> structFieldObjectInspectors) {
+		super(structFieldNames, structFieldObjectInspectors);
+	}
 
-    public LazyColumnarObjectInspector(List<StructField> fields) {
-        super(fields);
-    }
+	public LazyColumnarObjectInspector(List<StructField> fields) {
+		super(fields);
+	}
 
-    @Override
-    public Object getStructFieldData(Object data, StructField fieldRef) {
-        if (data == null) {
-            return null;
-        }
-        LazyColumnar struct = (LazyColumnar) data;
-        MyField f = (MyField) fieldRef;
+	@Override
+	public Object getStructFieldData(Object data, StructField fieldRef) {
+		if (data == null) {
+			return null;
+		}
+		LazyColumnar struct = (LazyColumnar) data;
+		MyField f = (MyField) fieldRef;
 
-        int fieldID = f.getFieldID();
-        assert (fieldID >= 0 && fieldID < fields.size());
+		int fieldID = f.getFieldID();
+		assert (fieldID >= 0 && fieldID < fields.size());
 
-        Object column = struct.getField(fieldID);
-        return column;
-    }
+		Object column = struct.getField(fieldID);
+		return column;
+	}
 
-    @Override
-    public List<Object> getStructFieldsDataAsList(Object data) {
-        if (data == null) {
-            return null;
-        }
-        LazyColumnar struct = (LazyColumnar) data;
-        return struct.getFieldsAsList();
-    }
+	@Override
+	public List<Object> getStructFieldsDataAsList(Object data) {
+		if (data == null) {
+			return null;
+		}
+		LazyColumnar struct = (LazyColumnar) data;
+		return struct.getFieldsAsList();
+	}
 
-    public String toString() {
-        String str = "";
-        for (MyField f : fields) {
-            str += f.getFieldName() + ":" + f.getFieldObjectInspector().getTypeName() + "  ";
-        }
-        return str;
-    }
+	public String toString() {
+		String str = "";
+		for (MyField f : fields) {
+			str += f.getFieldName() + ":"
+					+ f.getFieldObjectInspector().getTypeName() + "  ";
+		}
+		return str;
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java
index aaa5d66..dc4e85b 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyListObjectInspector.java
@@ -29,34 +29,34 @@
  */
 public class LazyListObjectInspector extends StandardListObjectInspector {
 
-    protected LazyListObjectInspector(ObjectInspector listElementObjectInspector) {
-        super(listElementObjectInspector);
-    }
+	protected LazyListObjectInspector(ObjectInspector listElementObjectInspector) {
+		super(listElementObjectInspector);
+	}
 
-    @Override
-    public List<?> getList(Object data) {
-        if (data == null) {
-            return null;
-        }
-        LazyArray array = (LazyArray) data;
-        return array.getList();
-    }
+	@Override
+	public List<?> getList(Object data) {
+		if (data == null) {
+			return null;
+		}
+		LazyArray array = (LazyArray) data;
+		return array.getList();
+	}
 
-    @Override
-    public Object getListElement(Object data, int index) {
-        if (data == null) {
-            return null;
-        }
-        LazyArray array = (LazyArray) data;
-        return array.getListElementObject(index);
-    }
+	@Override
+	public Object getListElement(Object data, int index) {
+		if (data == null) {
+			return null;
+		}
+		LazyArray array = (LazyArray) data;
+		return array.getListElementObject(index);
+	}
 
-    @Override
-    public int getListLength(Object data) {
-        if (data == null) {
-            return -1;
-        }
-        LazyArray array = (LazyArray) data;
-        return array.getListLength();
-    }
+	@Override
+	public int getListLength(Object data) {
+		if (data == null) {
+			return -1;
+		}
+		LazyArray array = (LazyArray) data;
+		return array.getListLength();
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java
index 1b0c412..a3be142 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyMapObjectInspector.java
@@ -31,31 +31,32 @@
  */
 public class LazyMapObjectInspector extends StandardMapObjectInspector {
 
-    protected LazyMapObjectInspector(ObjectInspector mapKeyObjectInspector, ObjectInspector mapValueObjectInspector) {
-        super(mapKeyObjectInspector, mapValueObjectInspector);
-    }
+	protected LazyMapObjectInspector(ObjectInspector mapKeyObjectInspector,
+			ObjectInspector mapValueObjectInspector) {
+		super(mapKeyObjectInspector, mapValueObjectInspector);
+	}
 
-    @Override
-    public Map<?, ?> getMap(Object data) {
-        if (data == null) {
-            return null;
-        }
-        return ((LazyMap) data).getMap();
-    }
+	@Override
+	public Map<?, ?> getMap(Object data) {
+		if (data == null) {
+			return null;
+		}
+		return ((LazyMap) data).getMap();
+	}
 
-    @Override
-    public int getMapSize(Object data) {
-        if (data == null) {
-            return -1;
-        }
-        return ((LazyMap) data).getMapSize();
-    }
+	@Override
+	public int getMapSize(Object data) {
+		if (data == null) {
+			return -1;
+		}
+		return ((LazyMap) data).getMapSize();
+	}
 
-    @Override
-    public Object getMapValueElement(Object data, Object key) {
-        if (data == null) {
-            return -1;
-        }
-        return ((LazyMap) data).getMapValueElement(key);
-    }
+	@Override
+	public Object getMapValueElement(Object data, Object key) {
+		if (data == null) {
+			return -1;
+		}
+		return ((LazyMap) data).getMapValueElement(key);
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java
index f57f6d9..439b130 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyObjectInspectorFactory.java
@@ -20,65 +20,74 @@
 
 public final class LazyObjectInspectorFactory {
 
-    static ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector> cachedLazyColumnarObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector>();
+	static ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector> cachedLazyColumnarObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyColumnarObjectInspector>();
 
-    static ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector> cachedLazyStructObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector>();
+	static ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector> cachedLazyStructObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStructObjectInspector>();
 
-    static ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector> cachedLazyListObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector>();
+	static ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector> cachedLazyListObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector>();
 
-    static ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector> cachedLazyMapObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector>();
+	static ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector> cachedLazyMapObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector>();
 
-    public static LazyColumnarObjectInspector getLazyColumnarObjectInspector(List<String> structFieldNames,
-            List<ObjectInspector> structFieldObjectInspectors) {
-        ArrayList<Object> signature = new ArrayList<Object>();
-        signature.add(structFieldNames);
-        signature.add(structFieldObjectInspectors);
-        LazyColumnarObjectInspector result = cachedLazyColumnarObjectInspector.get(signature);
-        if (result == null) {
-            result = new LazyColumnarObjectInspector(structFieldNames, structFieldObjectInspectors);
-            cachedLazyColumnarObjectInspector.put(signature, result);
-        }
-        return result;
-    }
+	public static LazyColumnarObjectInspector getLazyColumnarObjectInspector(
+			List<String> structFieldNames,
+			List<ObjectInspector> structFieldObjectInspectors) {
+		ArrayList<Object> signature = new ArrayList<Object>();
+		signature.add(structFieldNames);
+		signature.add(structFieldObjectInspectors);
+		LazyColumnarObjectInspector result = cachedLazyColumnarObjectInspector
+				.get(signature);
+		if (result == null) {
+			result = new LazyColumnarObjectInspector(structFieldNames,
+					structFieldObjectInspectors);
+			cachedLazyColumnarObjectInspector.put(signature, result);
+		}
+		return result;
+	}
 
-    public static LazyStructObjectInspector getLazyStructObjectInspector(List<String> structFieldNames,
-            List<ObjectInspector> structFieldObjectInspectors) {
-        ArrayList<Object> signature = new ArrayList<Object>();
-        signature.add(structFieldNames);
-        signature.add(structFieldObjectInspectors);
-        LazyStructObjectInspector result = cachedLazyStructObjectInspector.get(signature);
-        if (result == null) {
-            result = new LazyStructObjectInspector(structFieldNames, structFieldObjectInspectors);
-            cachedLazyStructObjectInspector.put(signature, result);
-        }
-        return result;
-    }
+	public static LazyStructObjectInspector getLazyStructObjectInspector(
+			List<String> structFieldNames,
+			List<ObjectInspector> structFieldObjectInspectors) {
+		ArrayList<Object> signature = new ArrayList<Object>();
+		signature.add(structFieldNames);
+		signature.add(structFieldObjectInspectors);
+		LazyStructObjectInspector result = cachedLazyStructObjectInspector
+				.get(signature);
+		if (result == null) {
+			result = new LazyStructObjectInspector(structFieldNames,
+					structFieldObjectInspectors);
+			cachedLazyStructObjectInspector.put(signature, result);
+		}
+		return result;
+	}
 
-    public static LazyListObjectInspector getLazyListObjectInspector(ObjectInspector listElementInspector) {
-        ArrayList<Object> signature = new ArrayList<Object>();
-        signature.add(listElementInspector);
-        LazyListObjectInspector result = cachedLazyListObjectInspector.get(signature);
-        if (result == null) {
-            result = new LazyListObjectInspector(listElementInspector);
-            cachedLazyListObjectInspector.put(signature, result);
-        }
-        return result;
-    }
+	public static LazyListObjectInspector getLazyListObjectInspector(
+			ObjectInspector listElementInspector) {
+		ArrayList<Object> signature = new ArrayList<Object>();
+		signature.add(listElementInspector);
+		LazyListObjectInspector result = cachedLazyListObjectInspector
+				.get(signature);
+		if (result == null) {
+			result = new LazyListObjectInspector(listElementInspector);
+			cachedLazyListObjectInspector.put(signature, result);
+		}
+		return result;
+	}
 
-    public static LazyMapObjectInspector getLazyMapObjectInspector(ObjectInspector keyInspector,
-            ObjectInspector valueInspector) {
-        ArrayList<Object> signature = new ArrayList<Object>();
-        signature.add(keyInspector);
-        signature.add(valueInspector);
-        LazyMapObjectInspector result = cachedLazyMapObjectInspector.get(signature);
-        if (result == null) {
-            result = new LazyMapObjectInspector(keyInspector, valueInspector);
-            cachedLazyMapObjectInspector.put(signature, result);
-        }
-        return result;
-    }
+	public static LazyMapObjectInspector getLazyMapObjectInspector(
+			ObjectInspector keyInspector, ObjectInspector valueInspector) {
+		ArrayList<Object> signature = new ArrayList<Object>();
+		signature.add(keyInspector);
+		signature.add(valueInspector);
+		LazyMapObjectInspector result = cachedLazyMapObjectInspector
+				.get(signature);
+		if (result == null) {
+			result = new LazyMapObjectInspector(keyInspector, valueInspector);
+			cachedLazyMapObjectInspector.put(signature, result);
+		}
+		return result;
+	}
 
-    private LazyObjectInspectorFactory() {
-        // prevent instantiation
-    }
+	private LazyObjectInspectorFactory() {
+		// prevent instantiation
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java
index ad70d4c..1a50233 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/LazyStructObjectInspector.java
@@ -32,34 +32,35 @@
  */
 public class LazyStructObjectInspector extends StandardStructObjectInspector {
 
-    protected LazyStructObjectInspector(List<String> structFieldNames, List<ObjectInspector> structFieldObjectInspectors) {
-        super(structFieldNames, structFieldObjectInspectors);
-    }
+	protected LazyStructObjectInspector(List<String> structFieldNames,
+			List<ObjectInspector> structFieldObjectInspectors) {
+		super(structFieldNames, structFieldObjectInspectors);
+	}
 
-    protected LazyStructObjectInspector(List<StructField> fields) {
-        super(fields);
-    }
+	protected LazyStructObjectInspector(List<StructField> fields) {
+		super(fields);
+	}
 
-    @Override
-    public Object getStructFieldData(Object data, StructField fieldRef) {
-        if (data == null) {
-            return null;
-        }
-        LazyStruct struct = (LazyStruct) data;
-        MyField f = (MyField) fieldRef;
+	@Override
+	public Object getStructFieldData(Object data, StructField fieldRef) {
+		if (data == null) {
+			return null;
+		}
+		LazyStruct struct = (LazyStruct) data;
+		MyField f = (MyField) fieldRef;
 
-        int fieldID = f.getFieldID();
-        assert (fieldID >= 0 && fieldID < fields.size());
+		int fieldID = f.getFieldID();
+		assert (fieldID >= 0 && fieldID < fields.size());
 
-        return struct.getField(fieldID);
-    }
+		return struct.getField(fieldID);
+	}
 
-    @Override
-    public List<Object> getStructFieldsDataAsList(Object data) {
-        if (data == null) {
-            return null;
-        }
-        LazyStruct struct = (LazyStruct) data;
-        return struct.getFieldsAsList();
-    }
+	@Override
+	public List<Object> getStructFieldsDataAsList(Object data) {
+		if (data == null) {
+			return null;
+		}
+		LazyStruct struct = (LazyStruct) data;
+		return struct.getFieldsAsList();
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java
index 6c19a73..7ef8bdd 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/AbstractPrimitiveLazyObjectInspector.java
@@ -26,22 +26,24 @@
 /**
  * An AbstractPrimitiveLazyObjectInspector for a LazyPrimitive object.
  */
-public abstract class AbstractPrimitiveLazyObjectInspector<T extends Writable> extends AbstractPrimitiveObjectInspector {
+public abstract class AbstractPrimitiveLazyObjectInspector<T extends Writable>
+		extends AbstractPrimitiveObjectInspector {
 
-    protected AbstractPrimitiveLazyObjectInspector(PrimitiveTypeEntry typeEntry) {
-        super(typeEntry);
-    }
+	protected AbstractPrimitiveLazyObjectInspector(PrimitiveTypeEntry typeEntry) {
+		super(typeEntry);
+	}
 
-    @Override
-    public T getPrimitiveWritableObject(Object o) {
-        if(o==null)
-            System.out.println("sth. wrong");
-        return o == null ? null : ((LazyPrimitive<?, T>) o).getWritableObject();
-    }
+	@SuppressWarnings("unchecked")
+	@Override
+	public T getPrimitiveWritableObject(Object o) {
+		if (o == null)
+			System.out.println("sth. wrong");
+		return o == null ? null : ((LazyPrimitive<?, T>) o).getWritableObject();
+	}
 
-    @Override
-    public boolean preferWritable() {
-        return true;
-    }
+	@Override
+	public boolean preferWritable() {
+		return true;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java
index 7927c1e..472dce0 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyBooleanObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableBooleanObjectInspector inspects a BooleanWritable Object.
  */
-public class LazyBooleanObjectInspector extends AbstractPrimitiveLazyObjectInspector<BooleanWritable> implements
-        BooleanObjectInspector {
+public class LazyBooleanObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<BooleanWritable> implements
+		BooleanObjectInspector {
 
-    LazyBooleanObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.booleanTypeEntry);
-    }
+	LazyBooleanObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.booleanTypeEntry);
+	}
 
-    @Override
-    public boolean get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public boolean get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyBoolean((LazyBoolean) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyBoolean((LazyBoolean) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Boolean.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Boolean.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java
index 10a881c..e631fc7 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyByteObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableByteObjectInspector inspects a ByteWritable Object.
  */
-public class LazyByteObjectInspector extends AbstractPrimitiveLazyObjectInspector<ByteWritable> implements
-        ByteObjectInspector {
+public class LazyByteObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<ByteWritable> implements
+		ByteObjectInspector {
 
-    LazyByteObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.byteTypeEntry);
-    }
+	LazyByteObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.byteTypeEntry);
+	}
 
-    @Override
-    public byte get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public byte get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyByte((LazyByte) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyByte((LazyByte) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Byte.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Byte.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java
index 9f98b56..1257f11 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyDoubleObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableDoubleObjectInspector inspects a DoubleWritable Object.
  */
-public class LazyDoubleObjectInspector extends AbstractPrimitiveLazyObjectInspector<DoubleWritable> implements
-        DoubleObjectInspector {
+public class LazyDoubleObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<DoubleWritable> implements
+		DoubleObjectInspector {
 
-    LazyDoubleObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.doubleTypeEntry);
-    }
+	LazyDoubleObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.doubleTypeEntry);
+	}
 
-    @Override
-    public double get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public double get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyDouble((LazyDouble) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyDouble((LazyDouble) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Double.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Double.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java
index bf3e9a2..c66a06f 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyFloatObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A FloatObjectInspector inspects a FloatWritable Object.
  */
-public class LazyFloatObjectInspector extends AbstractPrimitiveLazyObjectInspector<FloatWritable> implements
-        FloatObjectInspector {
+public class LazyFloatObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<FloatWritable> implements
+		FloatObjectInspector {
 
-    LazyFloatObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.floatTypeEntry);
-    }
+	LazyFloatObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.floatTypeEntry);
+	}
 
-    @Override
-    public float get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public float get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyFloat((LazyFloat) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyFloat((LazyFloat) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Float.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Float.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java
index 87bcb0d..b2159e0 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyIntObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableIntObjectInspector inspects a IntWritable Object.
  */
-public class LazyIntObjectInspector extends AbstractPrimitiveLazyObjectInspector<IntWritable> implements
-        IntObjectInspector {
+public class LazyIntObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<IntWritable> implements
+		IntObjectInspector {
 
-    LazyIntObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.intTypeEntry);
-    }
+	LazyIntObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.intTypeEntry);
+	}
 
-    @Override
-    public int get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public int get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyInteger((LazyInteger) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyInteger((LazyInteger) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Integer.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Integer.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java
index 06b5d3c..1fc2d53 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyLongObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableLongObjectInspector inspects a LongWritable Object.
  */
-public class LazyLongObjectInspector extends AbstractPrimitiveLazyObjectInspector<LongWritable> implements
-        LongObjectInspector {
+public class LazyLongObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<LongWritable> implements
+		LongObjectInspector {
 
-    LazyLongObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.longTypeEntry);
-    }
+	LazyLongObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.longTypeEntry);
+	}
 
-    @Override
-    public long get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public long get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyLong((LazyLong) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyLong((LazyLong) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Long.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Long.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
index c13b547..134dc5a 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
@@ -37,58 +37,63 @@
  */
 public final class LazyPrimitiveObjectInspectorFactory {
 
-    public static final LazyBooleanObjectInspector LAZY_BOOLEAN_OBJECT_INSPECTOR = new LazyBooleanObjectInspector();
-    public static final LazyByteObjectInspector LAZY_BYTE_OBJECT_INSPECTOR = new LazyByteObjectInspector();
-    public static final LazyShortObjectInspector LAZY_SHORT_OBJECT_INSPECTOR = new LazyShortObjectInspector();
-    public static final LazyIntObjectInspector LAZY_INT_OBJECT_INSPECTOR = new LazyIntObjectInspector();
-    public static final LazyLongObjectInspector LAZY_LONG_OBJECT_INSPECTOR = new LazyLongObjectInspector();
-    public static final LazyFloatObjectInspector LAZY_FLOAT_OBJECT_INSPECTOR = new LazyFloatObjectInspector();
-    public static final LazyDoubleObjectInspector LAZY_DOUBLE_OBJECT_INSPECTOR = new LazyDoubleObjectInspector();
-    public static final LazyVoidObjectInspector LAZY_VOID_OBJECT_INSPECTOR = new LazyVoidObjectInspector();
+	public static final LazyBooleanObjectInspector LAZY_BOOLEAN_OBJECT_INSPECTOR = new LazyBooleanObjectInspector();
+	public static final LazyByteObjectInspector LAZY_BYTE_OBJECT_INSPECTOR = new LazyByteObjectInspector();
+	public static final LazyShortObjectInspector LAZY_SHORT_OBJECT_INSPECTOR = new LazyShortObjectInspector();
+	public static final LazyIntObjectInspector LAZY_INT_OBJECT_INSPECTOR = new LazyIntObjectInspector();
+	public static final LazyLongObjectInspector LAZY_LONG_OBJECT_INSPECTOR = new LazyLongObjectInspector();
+	public static final LazyFloatObjectInspector LAZY_FLOAT_OBJECT_INSPECTOR = new LazyFloatObjectInspector();
+	public static final LazyDoubleObjectInspector LAZY_DOUBLE_OBJECT_INSPECTOR = new LazyDoubleObjectInspector();
+	public static final LazyVoidObjectInspector LAZY_VOID_OBJECT_INSPECTOR = new LazyVoidObjectInspector();
 
-    static ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector> cachedLazyStringObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector>();
+	static ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector> cachedLazyStringObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyStringObjectInspector>();
 
-    public static LazyStringObjectInspector getLazyStringObjectInspector(boolean escaped, byte escapeChar) {
-        ArrayList<Object> signature = new ArrayList<Object>();
-        signature.add(Boolean.valueOf(escaped));
-        signature.add(Byte.valueOf(escapeChar));
-        LazyStringObjectInspector result = cachedLazyStringObjectInspector.get(signature);
-        if (result == null) {
-            result = new LazyStringObjectInspector(escaped, escapeChar);
-            cachedLazyStringObjectInspector.put(signature, result);
-        }
-        return result;
-    }
+	public static LazyStringObjectInspector getLazyStringObjectInspector(
+			boolean escaped, byte escapeChar) {
+		ArrayList<Object> signature = new ArrayList<Object>();
+		signature.add(Boolean.valueOf(escaped));
+		signature.add(Byte.valueOf(escapeChar));
+		LazyStringObjectInspector result = cachedLazyStringObjectInspector
+				.get(signature);
+		if (result == null) {
+			result = new LazyStringObjectInspector(escaped, escapeChar);
+			cachedLazyStringObjectInspector.put(signature, result);
+		}
+		return result;
+	}
 
-    public static AbstractPrimitiveLazyObjectInspector<?> getLazyObjectInspector(PrimitiveCategory primitiveCategory,
-            boolean escaped, byte escapeChar) {
+	public static AbstractPrimitiveLazyObjectInspector<?> getLazyObjectInspector(
+			PrimitiveCategory primitiveCategory, boolean escaped,
+			byte escapeChar) {
 
-        switch (primitiveCategory) {
-            case BOOLEAN:
-                return LAZY_BOOLEAN_OBJECT_INSPECTOR;
-            case BYTE:
-                return LAZY_BYTE_OBJECT_INSPECTOR;
-            case SHORT:
-                return LAZY_SHORT_OBJECT_INSPECTOR;
-            case INT:
-                return LAZY_INT_OBJECT_INSPECTOR;
-            case LONG:
-                return LAZY_LONG_OBJECT_INSPECTOR;
-            case FLOAT:
-                return LAZY_FLOAT_OBJECT_INSPECTOR;
-            case DOUBLE:
-                return LAZY_DOUBLE_OBJECT_INSPECTOR;
-            case STRING:
-                return getLazyStringObjectInspector(escaped, escapeChar);
-            case VOID:
-                return LAZY_VOID_OBJECT_INSPECTOR;
-            default:
-                throw new RuntimeException("Internal error: Cannot find ObjectInspector " + " for " + primitiveCategory);
-        }
-    }
+		switch (primitiveCategory) {
+		case BOOLEAN:
+			return LAZY_BOOLEAN_OBJECT_INSPECTOR;
+		case BYTE:
+			return LAZY_BYTE_OBJECT_INSPECTOR;
+		case SHORT:
+			return LAZY_SHORT_OBJECT_INSPECTOR;
+		case INT:
+			return LAZY_INT_OBJECT_INSPECTOR;
+		case LONG:
+			return LAZY_LONG_OBJECT_INSPECTOR;
+		case FLOAT:
+			return LAZY_FLOAT_OBJECT_INSPECTOR;
+		case DOUBLE:
+			return LAZY_DOUBLE_OBJECT_INSPECTOR;
+		case STRING:
+			return getLazyStringObjectInspector(escaped, escapeChar);
+		case VOID:
+			return LAZY_VOID_OBJECT_INSPECTOR;
+		default:
+			throw new RuntimeException(
+					"Internal error: Cannot find ObjectInspector " + " for "
+							+ primitiveCategory);
+		}
+	}
 
-    private LazyPrimitiveObjectInspectorFactory() {
-        // prevent instantiation
-    }
+	private LazyPrimitiveObjectInspectorFactory() {
+		// prevent instantiation
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java
index b02d9bc..cb06dfd 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyShortObjectInspector.java
@@ -26,25 +26,26 @@
 /**
  * A WritableShortObjectInspector inspects a ShortWritable Object.
  */
-public class LazyShortObjectInspector extends AbstractPrimitiveLazyObjectInspector<ShortWritable> implements
-        ShortObjectInspector {
+public class LazyShortObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<ShortWritable> implements
+		ShortObjectInspector {
 
-    LazyShortObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.shortTypeEntry);
-    }
+	LazyShortObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.shortTypeEntry);
+	}
 
-    @Override
-    public short get(Object o) {
-        return getPrimitiveWritableObject(o).get();
-    }
+	@Override
+	public short get(Object o) {
+		return getPrimitiveWritableObject(o).get();
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyShort((LazyShort) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyShort((LazyShort) o);
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        return o == null ? null : Short.valueOf(get(o));
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		return o == null ? null : Short.valueOf(get(o));
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java
index 4d649dc..5832f34 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyStringObjectInspector.java
@@ -26,39 +26,41 @@
 /**
  * A WritableStringObjectInspector inspects a Text Object.
  */
-public class LazyStringObjectInspector extends AbstractPrimitiveLazyObjectInspector<Text> implements
-        StringObjectInspector {
+public class LazyStringObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<Text> implements
+		StringObjectInspector {
 
-    boolean escaped;
-    byte escapeChar;
+	boolean escaped;
+	byte escapeChar;
 
-    LazyStringObjectInspector(boolean escaped, byte escapeChar) {
-        super(PrimitiveObjectInspectorUtils.stringTypeEntry);
-        this.escaped = escaped;
-        this.escapeChar = escapeChar;
-    }
+	LazyStringObjectInspector(boolean escaped, byte escapeChar) {
+		super(PrimitiveObjectInspectorUtils.stringTypeEntry);
+		this.escaped = escaped;
+		this.escapeChar = escapeChar;
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o == null ? null : new LazyString((LazyString) o);
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o == null ? null : new LazyString((LazyString) o);
+	}
 
-    @Override
-    public Text getPrimitiveWritableObject(Object o) {
-        return o == null ? null : ((LazyString) o).getWritableObject();
-    }
+	@Override
+	public Text getPrimitiveWritableObject(Object o) {
+		return o == null ? null : ((LazyString) o).getWritableObject();
+	}
 
-    @Override
-    public String getPrimitiveJavaObject(Object o) {
-        return o == null ? null : ((LazyString) o).getWritableObject().toString();
-    }
+	@Override
+	public String getPrimitiveJavaObject(Object o) {
+		return o == null ? null : ((LazyString) o).getWritableObject()
+				.toString();
+	}
 
-    public boolean isEscaped() {
-        return escaped;
-    }
+	public boolean isEscaped() {
+		return escaped;
+	}
 
-    public byte getEscapeChar() {
-        return escapeChar;
-    }
+	public byte getEscapeChar() {
+		return escapeChar;
+	}
 
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java
index c916191..a30f1af 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/LazyVoidObjectInspector.java
@@ -24,20 +24,21 @@
 /**
  * A WritableVoidObjectInspector inspects a NullWritable Object.
  */
-public class LazyVoidObjectInspector extends AbstractPrimitiveLazyObjectInspector<NullWritable> implements
-        VoidObjectInspector {
+public class LazyVoidObjectInspector extends
+		AbstractPrimitiveLazyObjectInspector<NullWritable> implements
+		VoidObjectInspector {
 
-    LazyVoidObjectInspector() {
-        super(PrimitiveObjectInspectorUtils.voidTypeEntry);
-    }
+	LazyVoidObjectInspector() {
+		super(PrimitiveObjectInspectorUtils.voidTypeEntry);
+	}
 
-    @Override
-    public Object copyObject(Object o) {
-        return o;
-    }
+	@Override
+	public Object copyObject(Object o) {
+		return o;
+	}
 
-    @Override
-    public Object getPrimitiveJavaObject(Object o) {
-        throw new RuntimeException("Internal error: cannot create Void object.");
-    }
+	@Override
+	public Object getPrimitiveJavaObject(Object o) {
+		throw new RuntimeException("Internal error: cannot create Void object.");
+	}
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
index dbb766a..e70bdb9 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/lazy/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
@@ -32,46 +32,58 @@
  */

 public final class PrimitiveObjectInspectorFactory {

 

-    public static final LazyBooleanObjectInspector LazyBooleanObjectInspector = new LazyBooleanObjectInspector();

-    public static final LazyByteObjectInspector LazyByteObjectInspector = new LazyByteObjectInspector();

-    public static final LazyShortObjectInspector LazyShortObjectInspector = new LazyShortObjectInspector();

-    public static final LazyIntObjectInspector LazyIntObjectInspector = new LazyIntObjectInspector();

-    public static final LazyLongObjectInspector LazyLongObjectInspector = new LazyLongObjectInspector();

-    public static final LazyFloatObjectInspector LazyFloatObjectInspector = new LazyFloatObjectInspector();

-    public static final LazyDoubleObjectInspector LazyDoubleObjectInspector = new LazyDoubleObjectInspector();

-    public static final LazyStringObjectInspector LazyStringObjectInspector = new LazyStringObjectInspector(false,

-            (byte) '\\');

-    public static final LazyVoidObjectInspector LazyVoidObjectInspector = new LazyVoidObjectInspector();

+	public static final LazyBooleanObjectInspector LazyBooleanObjectInspector = new LazyBooleanObjectInspector();

+	public static final LazyByteObjectInspector LazyByteObjectInspector = new LazyByteObjectInspector();

+	public static final LazyShortObjectInspector LazyShortObjectInspector = new LazyShortObjectInspector();

+	public static final LazyIntObjectInspector LazyIntObjectInspector = new LazyIntObjectInspector();

+	public static final LazyLongObjectInspector LazyLongObjectInspector = new LazyLongObjectInspector();

+	public static final LazyFloatObjectInspector LazyFloatObjectInspector = new LazyFloatObjectInspector();

+	public static final LazyDoubleObjectInspector LazyDoubleObjectInspector = new LazyDoubleObjectInspector();

+	public static final LazyStringObjectInspector LazyStringObjectInspector = new LazyStringObjectInspector(

+			false, (byte) '\\');

+	public static final LazyVoidObjectInspector LazyVoidObjectInspector = new LazyVoidObjectInspector();

 

-    private static HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>> cachedPrimitiveLazyInspectorCache = new HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>>();

+	private static HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>> cachedPrimitiveLazyInspectorCache = new HashMap<PrimitiveCategory, AbstractPrimitiveLazyObjectInspector<?>>();

 

-    static {

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BOOLEAN, LazyBooleanObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BYTE, LazyByteObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.SHORT, LazyShortObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.INT, LazyIntObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.LONG, LazyLongObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.FLOAT, LazyFloatObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.DOUBLE, LazyDoubleObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.STRING, LazyStringObjectInspector);

-        cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.VOID, LazyVoidObjectInspector);

-    }

+	static {

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BOOLEAN,

+				LazyBooleanObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.BYTE,

+				LazyByteObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.SHORT,

+				LazyShortObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.INT,

+				LazyIntObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.LONG,

+				LazyLongObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.FLOAT,

+				LazyFloatObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.DOUBLE,

+				LazyDoubleObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.STRING,

+				LazyStringObjectInspector);

+		cachedPrimitiveLazyInspectorCache.put(PrimitiveCategory.VOID,

+				LazyVoidObjectInspector);

+	}

 

-    /**

-     * Returns the PrimitiveWritableObjectInspector for the PrimitiveCategory.

-     * 

-     * @param primitiveCategory

-     */

-    public static AbstractPrimitiveLazyObjectInspector<?> getPrimitiveLazyObjectInspector(

-            PrimitiveCategory primitiveCategory) {

-        AbstractPrimitiveLazyObjectInspector<?> result = cachedPrimitiveLazyInspectorCache.get(primitiveCategory);

-        if (result == null) {

-            throw new RuntimeException("Internal error: Cannot find ObjectInspector " + " for " + primitiveCategory);

-        }

-        return result;

-    }

+	/**

+	 * Returns the PrimitiveWritableObjectInspector for the PrimitiveCategory.

+	 * 

+	 * @param primitiveCategory

+	 */

+	public static AbstractPrimitiveLazyObjectInspector<?> getPrimitiveLazyObjectInspector(

+			PrimitiveCategory primitiveCategory) {

+		AbstractPrimitiveLazyObjectInspector<?> result = cachedPrimitiveLazyInspectorCache

+				.get(primitiveCategory);

+		if (result == null) {

+			throw new RuntimeException(

+					"Internal error: Cannot find ObjectInspector " + " for "

+							+ primitiveCategory);

+		}

+		return result;

+	}

 

-    private PrimitiveObjectInspectorFactory() {

-        // prevent instantiation

-    }

+	private PrimitiveObjectInspectorFactory() {

+		// prevent instantiation

+	}

 }

diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java
index 7830c52..aeea68f 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/IHiveParser.java
@@ -5,12 +5,13 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
 public interface IHiveParser {
-    /**
-     * parse one hive rwo into
-     * 
-     * @param row
-     * @param objectInspector
-     * @param tb
-     */
-    public void parse(byte[] data, int start, int length, ArrayTupleBuilder tb) throws IOException;
+	/**
+	 * parse one hive rwo into
+	 * 
+	 * @param row
+	 * @param objectInspector
+	 * @param tb
+	 */
+	public void parse(byte[] data, int start, int length, ArrayTupleBuilder tb)
+			throws IOException;
 }
diff --git a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java
index 38e1b36..3aeb058 100644
--- a/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java
+++ b/hivesterix/src/main/java/edu/uci/ics/hivesterix/serde/parser/TextToBinaryTupleParser.java
@@ -19,156 +19,166 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
 public class TextToBinaryTupleParser implements IHiveParser {
-    private int[] invertedIndex;
-    private int[] fieldEnds;
-    private int lastNecessaryFieldIndex;
-    private LazySimpleStructObjectInspector inputObjectInspector;
-    private List<? extends StructField> fieldRefs;
+	private int[] invertedIndex;
+	private int[] fieldEnds;
+	private int lastNecessaryFieldIndex;
+	private LazySimpleStructObjectInspector inputObjectInspector;
+	private List<? extends StructField> fieldRefs;
 
-    public TextToBinaryTupleParser(int[] outputColumnsOffset, ObjectInspector structInspector) {
-        int size = 0;
-        for (int i = 0; i < outputColumnsOffset.length; i++)
-            if (outputColumnsOffset[i] >= 0)
-                size++;
-        invertedIndex = new int[size];
-        for (int i = 0; i < outputColumnsOffset.length; i++)
-            if (outputColumnsOffset[i] >= 0) {
-                invertedIndex[outputColumnsOffset[i]] = i;
-                lastNecessaryFieldIndex = i;
-            }
-        fieldEnds = new int[outputColumnsOffset.length];
-        for (int i = 0; i < fieldEnds.length; i++)
-            fieldEnds[i] = 0;
-        inputObjectInspector = (LazySimpleStructObjectInspector) structInspector;
-        fieldRefs = inputObjectInspector.getAllStructFieldRefs();
-    }
+	public TextToBinaryTupleParser(int[] outputColumnsOffset,
+			ObjectInspector structInspector) {
+		int size = 0;
+		for (int i = 0; i < outputColumnsOffset.length; i++)
+			if (outputColumnsOffset[i] >= 0)
+				size++;
+		invertedIndex = new int[size];
+		for (int i = 0; i < outputColumnsOffset.length; i++)
+			if (outputColumnsOffset[i] >= 0) {
+				invertedIndex[outputColumnsOffset[i]] = i;
+				lastNecessaryFieldIndex = i;
+			}
+		fieldEnds = new int[outputColumnsOffset.length];
+		for (int i = 0; i < fieldEnds.length; i++)
+			fieldEnds[i] = 0;
+		inputObjectInspector = (LazySimpleStructObjectInspector) structInspector;
+		fieldRefs = inputObjectInspector.getAllStructFieldRefs();
+	}
 
-    @Override
-    public void parse(byte[] bytes, int start, int length, ArrayTupleBuilder tb) throws IOException {
-        byte separator = inputObjectInspector.getSeparator();
-        boolean lastColumnTakesRest = inputObjectInspector.getLastColumnTakesRest();
-        boolean isEscaped = inputObjectInspector.isEscaped();
-        byte escapeChar = inputObjectInspector.getEscapeChar();
-        DataOutput output = tb.getDataOutput();
+	@Override
+	public void parse(byte[] bytes, int start, int length, ArrayTupleBuilder tb)
+			throws IOException {
+		byte separator = inputObjectInspector.getSeparator();
+		boolean lastColumnTakesRest = inputObjectInspector
+				.getLastColumnTakesRest();
+		boolean isEscaped = inputObjectInspector.isEscaped();
+		byte escapeChar = inputObjectInspector.getEscapeChar();
+		DataOutput output = tb.getDataOutput();
 
-        int structByteEnd = start + length - 1;
-        int fieldId = 0;
-        int fieldByteEnd = start;
+		int structByteEnd = start + length - 1;
+		int fieldId = 0;
+		int fieldByteEnd = start;
 
-        // Go through all bytes in the byte[]
-        while (fieldByteEnd <= structByteEnd && fieldId <= lastNecessaryFieldIndex) {
-            if (fieldByteEnd == structByteEnd || bytes[fieldByteEnd] == separator) {
-                // Reached the end of a field?
-                if (lastColumnTakesRest && fieldId == fieldEnds.length - 1) {
-                    fieldByteEnd = structByteEnd;
-                }
-                fieldEnds[fieldId] = fieldByteEnd;
-                if (fieldId == fieldEnds.length - 1 || fieldByteEnd == structByteEnd) {
-                    // for the case of null fields
-                    for (int i = fieldId; i < fieldEnds.length; i++) {
-                        fieldEnds[i] = fieldByteEnd;
-                    }
-                    break;
-                }
-                fieldByteEnd++;
-                fieldId++;
-            } else {
-                if (isEscaped && bytes[fieldByteEnd] == escapeChar && fieldByteEnd + 1 < structByteEnd) {
-                    // ignore the char after escape_char
-                    fieldByteEnd += 2;
-                } else {
-                    fieldByteEnd++;
-                }
-            }
-        }
+		// Go through all bytes in the byte[]
+		while (fieldByteEnd <= structByteEnd
+				&& fieldId <= lastNecessaryFieldIndex) {
+			if (fieldByteEnd == structByteEnd
+					|| bytes[fieldByteEnd] == separator) {
+				// Reached the end of a field?
+				if (lastColumnTakesRest && fieldId == fieldEnds.length - 1) {
+					fieldByteEnd = structByteEnd;
+				}
+				fieldEnds[fieldId] = fieldByteEnd;
+				if (fieldId == fieldEnds.length - 1
+						|| fieldByteEnd == structByteEnd) {
+					// for the case of null fields
+					for (int i = fieldId; i < fieldEnds.length; i++) {
+						fieldEnds[i] = fieldByteEnd;
+					}
+					break;
+				}
+				fieldByteEnd++;
+				fieldId++;
+			} else {
+				if (isEscaped && bytes[fieldByteEnd] == escapeChar
+						&& fieldByteEnd + 1 < structByteEnd) {
+					// ignore the char after escape_char
+					fieldByteEnd += 2;
+				} else {
+					fieldByteEnd++;
+				}
+			}
+		}
 
-        for (int i = 0; i < invertedIndex.length; i++) {
-            int index = invertedIndex[i];
-            StructField fieldRef = fieldRefs.get(index);
-            ObjectInspector inspector = fieldRef.getFieldObjectInspector();
-            Category category = inspector.getCategory();
-            int fieldStart = index == 0 ? 0 : fieldEnds[index - 1] + 1;
-            int fieldEnd = fieldEnds[index];
-            if (bytes[fieldEnd] == separator)
-                fieldEnd--;
-            int fieldLen = fieldEnd - fieldStart + 1;
-            switch (category) {
-                case PRIMITIVE:
-                    PrimitiveObjectInspector poi = (PrimitiveObjectInspector) inspector;
-                    switch (poi.getPrimitiveCategory()) {
-                        case VOID: {
-                            break;
-                        }
-                        case BOOLEAN: {
-                            output.write(bytes[fieldStart]);
-                            break;
-                        }
-                        case BYTE: {
-                            output.write(bytes[fieldStart]);
-                            break;
-                        }
-                        case SHORT: {
-                            short v = LazyShort.parseShort(bytes, fieldStart, fieldLen);
-                            output.write((byte) (v >> 8));
-                            output.write((byte) (v));
-                            break;
-                        }
-                        case INT: {
-                            int v = LazyInteger.parseInt(bytes, fieldStart, fieldLen);
-                            LazyUtils.writeVInt(output, v);
-                            break;
-                        }
-                        case LONG: {
-                            long v = LazyLong.parseLong(bytes, fieldStart, fieldLen);
-                            LazyUtils.writeVLong(output, v);
-                            break;
-                        }
-                        case FLOAT: {
-                            float value = Float.parseFloat(Text.decode(bytes, fieldStart, fieldLen));
-                            int v = Float.floatToIntBits(value);
-                            output.write((byte) (v >> 24));
-                            output.write((byte) (v >> 16));
-                            output.write((byte) (v >> 8));
-                            output.write((byte) (v));
-                            break;
-                        }
-                        case DOUBLE: {
-                            try {
-                                double value = Double.parseDouble(Text.decode(bytes, fieldStart, fieldLen));
-                                long v = Double.doubleToLongBits(value);
-                                output.write((byte) (v >> 56));
-                                output.write((byte) (v >> 48));
-                                output.write((byte) (v >> 40));
-                                output.write((byte) (v >> 32));
-                                output.write((byte) (v >> 24));
-                                output.write((byte) (v >> 16));
-                                output.write((byte) (v >> 8));
-                                output.write((byte) (v));
-                            } catch (NumberFormatException e) {
-                                throw e;
-                            }
-                            break;
-                        }
-                        case STRING: {
-                            LazyUtils.writeVInt(output, fieldLen);
-                            output.write(bytes, fieldStart, fieldLen);
-                            break;
-                        }
-                        default: {
-                            throw new RuntimeException("Unrecognized type: " + poi.getPrimitiveCategory());
-                        }
-                    }
-                    break;
-                case STRUCT:
-                    throw new NotImplementedException("Unrecognized type: struct ");
-                case LIST:
-                    throw new NotImplementedException("Unrecognized type: struct ");
-                case MAP:
-                    throw new NotImplementedException("Unrecognized type: struct ");
-                case UNION:
-                    throw new NotImplementedException("Unrecognized type: struct ");
-            }
-            tb.addFieldEndOffset();
-        }
-    }
+		for (int i = 0; i < invertedIndex.length; i++) {
+			int index = invertedIndex[i];
+			StructField fieldRef = fieldRefs.get(index);
+			ObjectInspector inspector = fieldRef.getFieldObjectInspector();
+			Category category = inspector.getCategory();
+			int fieldStart = index == 0 ? 0 : fieldEnds[index - 1] + 1;
+			int fieldEnd = fieldEnds[index];
+			if (bytes[fieldEnd] == separator)
+				fieldEnd--;
+			int fieldLen = fieldEnd - fieldStart + 1;
+			switch (category) {
+			case PRIMITIVE:
+				PrimitiveObjectInspector poi = (PrimitiveObjectInspector) inspector;
+				switch (poi.getPrimitiveCategory()) {
+				case VOID: {
+					break;
+				}
+				case BOOLEAN: {
+					output.write(bytes[fieldStart]);
+					break;
+				}
+				case BYTE: {
+					output.write(bytes[fieldStart]);
+					break;
+				}
+				case SHORT: {
+					short v = LazyShort.parseShort(bytes, fieldStart, fieldLen);
+					output.write((byte) (v >> 8));
+					output.write((byte) (v));
+					break;
+				}
+				case INT: {
+					int v = LazyInteger.parseInt(bytes, fieldStart, fieldLen);
+					LazyUtils.writeVInt(output, v);
+					break;
+				}
+				case LONG: {
+					long v = LazyLong.parseLong(bytes, fieldStart, fieldLen);
+					LazyUtils.writeVLong(output, v);
+					break;
+				}
+				case FLOAT: {
+					float value = Float.parseFloat(Text.decode(bytes,
+							fieldStart, fieldLen));
+					int v = Float.floatToIntBits(value);
+					output.write((byte) (v >> 24));
+					output.write((byte) (v >> 16));
+					output.write((byte) (v >> 8));
+					output.write((byte) (v));
+					break;
+				}
+				case DOUBLE: {
+					try {
+						double value = Double.parseDouble(Text.decode(bytes,
+								fieldStart, fieldLen));
+						long v = Double.doubleToLongBits(value);
+						output.write((byte) (v >> 56));
+						output.write((byte) (v >> 48));
+						output.write((byte) (v >> 40));
+						output.write((byte) (v >> 32));
+						output.write((byte) (v >> 24));
+						output.write((byte) (v >> 16));
+						output.write((byte) (v >> 8));
+						output.write((byte) (v));
+					} catch (NumberFormatException e) {
+						throw e;
+					}
+					break;
+				}
+				case STRING: {
+					LazyUtils.writeVInt(output, fieldLen);
+					output.write(bytes, fieldStart, fieldLen);
+					break;
+				}
+				default: {
+					throw new RuntimeException("Unrecognized type: "
+							+ poi.getPrimitiveCategory());
+				}
+				}
+				break;
+			case STRUCT:
+				throw new NotImplementedException("Unrecognized type: struct ");
+			case LIST:
+				throw new NotImplementedException("Unrecognized type: struct ");
+			case MAP:
+				throw new NotImplementedException("Unrecognized type: struct ");
+			case UNION:
+				throw new NotImplementedException("Unrecognized type: struct ");
+			}
+			tb.addFieldEndOffset();
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/Driver.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/Driver.java
index de21cf9..57e2cc0 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/Driver.java
@@ -113,1199 +113,1329 @@
 import edu.uci.ics.hivesterix.runtime.exec.HyracksExecutionEngine;
 import edu.uci.ics.hivesterix.runtime.exec.IExecutionEngine;
 
+@SuppressWarnings({ "deprecation", "unused" })
 public class Driver implements CommandProcessor {
 
-    static final private Log LOG = LogFactory.getLog(Driver.class.getName());
-    static final private LogHelper console = new LogHelper(LOG);
+	static final private Log LOG = LogFactory.getLog(Driver.class.getName());
+	static final private LogHelper console = new LogHelper(LOG);
 
-    // hive-sterix
-    private IExecutionEngine engine;
-    private boolean hivesterix = false;
-    private PrintWriter planPrinter;
+	// hive-sterix
+	private IExecutionEngine engine;
+	private boolean hivesterix = false;
 
-    private int maxRows = 100;
-    ByteStream.Output bos = new ByteStream.Output();
+	private int maxRows = 100;
+	ByteStream.Output bos = new ByteStream.Output();
 
-    private HiveConf conf;
-    private DataInput resStream;
-    private Context ctx;
-    private QueryPlan plan;
-    private Schema schema;
-    private HiveLockManager hiveLockMgr;
+	private HiveConf conf;
+	private DataInput resStream;
+	private Context ctx;
+	private QueryPlan plan;
+	private Schema schema;
+	private HiveLockManager hiveLockMgr;
 
-    private String errorMessage;
-    private String SQLState;
+	private String errorMessage;
+	private String SQLState;
 
-    // A limit on the number of threads that can be launched
-    private int maxthreads;
-    private final int sleeptime = 2000;
+	// A limit on the number of threads that can be launched
+	private int maxthreads;
+	private final int sleeptime = 2000;
 
-    protected int tryCount = Integer.MAX_VALUE;
+	protected int tryCount = Integer.MAX_VALUE;
 
-    private int checkLockManager() {
-        boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
-        if (supportConcurrency && (hiveLockMgr == null)) {
-            try {
-                setLockManager();
-            } catch (SemanticException e) {
-                errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
-                SQLState = ErrorMsg.findSQLState(e.getMessage());
-                console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-                return (12);
-            }
-        }
-        return (0);
-    }
+	private int checkLockManager() {
+		boolean supportConcurrency = conf
+				.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+		if (supportConcurrency && (hiveLockMgr == null)) {
+			try {
+				setLockManager();
+			} catch (SemanticException e) {
+				errorMessage = "FAILED: Error in semantic analysis: "
+						+ e.getMessage();
+				SQLState = ErrorMsg.findSQLState(e.getMessage());
+				console.printError(
+						errorMessage,
+						"\n"
+								+ org.apache.hadoop.util.StringUtils
+										.stringifyException(e));
+				return (12);
+			}
+		}
+		return (0);
+	}
 
-    private void setLockManager() throws SemanticException {
-        boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
-        if (supportConcurrency) {
-            String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER);
-            if ((lockMgr == null) || (lockMgr.isEmpty())) {
-                throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg());
-            }
+	private void setLockManager() throws SemanticException {
+		boolean supportConcurrency = conf
+				.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+		if (supportConcurrency) {
+			String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER);
+			if ((lockMgr == null) || (lockMgr.isEmpty())) {
+				throw new SemanticException(
+						ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg());
+			}
 
-            try {
-                hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr), conf);
-                hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
-            } catch (Exception e) {
-                throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e.getMessage());
-            }
-        }
-    }
+			try {
+				hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(
+						conf.getClassByName(lockMgr), conf);
+				hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
+			} catch (Exception e) {
+				throw new SemanticException(
+						ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg()
+								+ e.getMessage());
+			}
+		}
+	}
 
-    public void init() {
-        Operator.resetId();
-    }
+	public void init() {
+		Operator.resetId();
+	}
 
-    /**
-     * Return the status information about the Map-Reduce cluster
-     */
-    public ClusterStatus getClusterStatus() throws Exception {
-        ClusterStatus cs;
-        try {
-            JobConf job = new JobConf(conf, ExecDriver.class);
-            JobClient jc = new JobClient(job);
-            cs = jc.getClusterStatus();
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-        LOG.info("Returning cluster status: " + cs.toString());
-        return cs;
-    }
+	/**
+	 * Return the status information about the Map-Reduce cluster
+	 */
+	public ClusterStatus getClusterStatus() throws Exception {
+		ClusterStatus cs;
+		try {
+			JobConf job = new JobConf(conf, ExecDriver.class);
+			JobClient jc = new JobClient(job);
+			cs = jc.getClusterStatus();
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw e;
+		}
+		LOG.info("Returning cluster status: " + cs.toString());
+		return cs;
+	}
 
-    public Schema getSchema() {
-        return schema;
-    }
+	public Schema getSchema() {
+		return schema;
+	}
 
-    /**
-     * Get a Schema with fields represented with native Hive types
-     */
-    public static Schema getSchema(BaseSemanticAnalyzer sem, HiveConf conf) {
-        Schema schema = null;
+	/**
+	 * Get a Schema with fields represented with native Hive types
+	 */
+	public static Schema getSchema(BaseSemanticAnalyzer sem, HiveConf conf) {
+		Schema schema = null;
 
-        // If we have a plan, prefer its logical result schema if it's
-        // available; otherwise, try digging out a fetch task; failing that,
-        // give up.
-        if (sem == null) {
-            // can't get any info without a plan
-        } else if (sem.getResultSchema() != null) {
-            List<FieldSchema> lst = sem.getResultSchema();
-            schema = new Schema(lst, null);
-        } else if (sem.getFetchTask() != null) {
-            FetchTask ft = sem.getFetchTask();
-            TableDesc td = ft.getTblDesc();
-            // partitioned tables don't have tableDesc set on the FetchTask.
-            // Instead
-            // they have a list of PartitionDesc objects, each with a table
-            // desc.
-            // Let's
-            // try to fetch the desc for the first partition and use it's
-            // deserializer.
-            if (td == null && ft.getWork() != null && ft.getWork().getPartDesc() != null) {
-                if (ft.getWork().getPartDesc().size() > 0) {
-                    td = ft.getWork().getPartDesc().get(0).getTableDesc();
-                }
-            }
+		// If we have a plan, prefer its logical result schema if it's
+		// available; otherwise, try digging out a fetch task; failing that,
+		// give up.
+		if (sem == null) {
+			// can't get any info without a plan
+		} else if (sem.getResultSchema() != null) {
+			List<FieldSchema> lst = sem.getResultSchema();
+			schema = new Schema(lst, null);
+		} else if (sem.getFetchTask() != null) {
+			FetchTask ft = sem.getFetchTask();
+			TableDesc td = ft.getTblDesc();
+			// partitioned tables don't have tableDesc set on the FetchTask.
+			// Instead
+			// they have a list of PartitionDesc objects, each with a table
+			// desc.
+			// Let's
+			// try to fetch the desc for the first partition and use it's
+			// deserializer.
+			if (td == null && ft.getWork() != null
+					&& ft.getWork().getPartDesc() != null) {
+				if (ft.getWork().getPartDesc().size() > 0) {
+					td = ft.getWork().getPartDesc().get(0).getTableDesc();
+				}
+			}
 
-            if (td == null) {
-                LOG.info("No returning schema.");
-            } else {
-                String tableName = "result";
-                List<FieldSchema> lst = null;
-                try {
-                    lst = MetaStoreUtils.getFieldsFromDeserializer(tableName, td.getDeserializer());
-                } catch (Exception e) {
-                    LOG.warn("Error getting schema: " + org.apache.hadoop.util.StringUtils.stringifyException(e));
-                }
-                if (lst != null) {
-                    schema = new Schema(lst, null);
-                }
-            }
-        }
-        if (schema == null) {
-            schema = new Schema();
-        }
-        LOG.info("Returning Hive schema: " + schema);
-        return schema;
-    }
+			if (td == null) {
+				LOG.info("No returning schema.");
+			} else {
+				String tableName = "result";
+				List<FieldSchema> lst = null;
+				try {
+					lst = MetaStoreUtils.getFieldsFromDeserializer(tableName,
+							td.getDeserializer());
+				} catch (Exception e) {
+					LOG.warn("Error getting schema: "
+							+ org.apache.hadoop.util.StringUtils
+									.stringifyException(e));
+				}
+				if (lst != null) {
+					schema = new Schema(lst, null);
+				}
+			}
+		}
+		if (schema == null) {
+			schema = new Schema();
+		}
+		LOG.info("Returning Hive schema: " + schema);
+		return schema;
+	}
 
-    /**
-     * Get a Schema with fields represented with Thrift DDL types
-     */
-    public Schema getThriftSchema() throws Exception {
-        Schema schema;
-        try {
-            schema = getSchema();
-            if (schema != null) {
-                List<FieldSchema> lst = schema.getFieldSchemas();
-                // Go over the schema and convert type to thrift type
-                if (lst != null) {
-                    for (FieldSchema f : lst) {
-                        f.setType(MetaStoreUtils.typeToThriftType(f.getType()));
-                    }
-                }
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-        LOG.info("Returning Thrift schema: " + schema);
-        return schema;
-    }
+	/**
+	 * Get a Schema with fields represented with Thrift DDL types
+	 */
+	public Schema getThriftSchema() throws Exception {
+		Schema schema;
+		try {
+			schema = getSchema();
+			if (schema != null) {
+				List<FieldSchema> lst = schema.getFieldSchemas();
+				// Go over the schema and convert type to thrift type
+				if (lst != null) {
+					for (FieldSchema f : lst) {
+						f.setType(MetaStoreUtils.typeToThriftType(f.getType()));
+					}
+				}
+			}
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw e;
+		}
+		LOG.info("Returning Thrift schema: " + schema);
+		return schema;
+	}
 
-    /**
-     * Return the maximum number of rows returned by getResults
-     */
-    public int getMaxRows() {
-        return maxRows;
-    }
+	/**
+	 * Return the maximum number of rows returned by getResults
+	 */
+	public int getMaxRows() {
+		return maxRows;
+	}
 
-    /**
-     * Set the maximum number of rows returned by getResults
-     */
-    public void setMaxRows(int maxRows) {
-        this.maxRows = maxRows;
-    }
+	/**
+	 * Set the maximum number of rows returned by getResults
+	 */
+	public void setMaxRows(int maxRows) {
+		this.maxRows = maxRows;
+	}
 
-    public boolean hasReduceTasks(List<Task<? extends Serializable>> tasks) {
-        if (tasks == null) {
-            return false;
-        }
+	public boolean hasReduceTasks(List<Task<? extends Serializable>> tasks) {
+		if (tasks == null) {
+			return false;
+		}
 
-        boolean hasReduce = false;
-        for (Task<? extends Serializable> task : tasks) {
-            if (task.hasReduce()) {
-                return true;
-            }
+		boolean hasReduce = false;
+		for (Task<? extends Serializable> task : tasks) {
+			if (task.hasReduce()) {
+				return true;
+			}
 
-            hasReduce = (hasReduce || hasReduceTasks(task.getChildTasks()));
-        }
-        return hasReduce;
-    }
+			hasReduce = (hasReduce || hasReduceTasks(task.getChildTasks()));
+		}
+		return hasReduce;
+	}
 
-    /**
-     * for backwards compatibility with current tests
-     */
-    public Driver(HiveConf conf) {
-        this.conf = conf;
+	/**
+	 * for backwards compatibility with current tests
+	 */
+	public Driver(HiveConf conf) {
+		this.conf = conf;
 
-        // hivesterix
-        engine = new HyracksExecutionEngine(conf);
-    }
+		// hivesterix
+		engine = new HyracksExecutionEngine(conf);
+	}
 
-    public Driver() {
-        if (SessionState.get() != null) {
-            conf = SessionState.get().getConf();
-        }
+	public Driver() {
+		if (SessionState.get() != null) {
+			conf = SessionState.get().getConf();
+		}
 
-        // hivesterix
-        engine = new HyracksExecutionEngine(conf);
-    }
+		// hivesterix
+		engine = new HyracksExecutionEngine(conf);
+	}
 
-    // hivesterix: plan printer
-    public Driver(HiveConf conf, PrintWriter planPrinter) {
-        this.conf = conf;
-        engine = new HyracksExecutionEngine(conf, planPrinter);
-        this.planPrinter = planPrinter;
-    }
+	// hivesterix: plan printer
+	public Driver(HiveConf conf, PrintWriter planPrinter) {
+		this.conf = conf;
+		engine = new HyracksExecutionEngine(conf, planPrinter);
+	}
 
-    public void clear() {
-        this.hivesterix = false;
-    }
+	public void clear() {
+		this.hivesterix = false;
+	}
 
-    /**
-     * Compile a new query. Any currently-planned query associated with this
-     * Driver is discarded.
-     * 
-     * @param command
-     *            The SQL query to compile.
-     */
-    public int compile(String command) {
-        if (plan != null) {
-            close();
-            plan = null;
-        }
+	/**
+	 * Compile a new query. Any currently-planned query associated with this
+	 * Driver is discarded.
+	 * 
+	 * @param command
+	 *            The SQL query to compile.
+	 */
+	public int compile(String command) {
+		if (plan != null) {
+			close();
+			plan = null;
+		}
 
-        TaskFactory.resetId();
+		TaskFactory.resetId();
 
-        try {
-            command = new VariableSubstitution().substitute(conf, command);
-            ctx = new Context(conf);
+		try {
+			command = new VariableSubstitution().substitute(conf, command);
+			ctx = new Context(conf);
 
-            ParseDriver pd = new ParseDriver();
-            ASTNode tree = pd.parse(command, ctx);
-            tree = ParseUtils.findRootNonNullToken(tree);
+			ParseDriver pd = new ParseDriver();
+			ASTNode tree = pd.parse(command, ctx);
+			tree = ParseUtils.findRootNonNullToken(tree);
 
-            BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
-            List<AbstractSemanticAnalyzerHook> saHooks = getSemanticAnalyzerHooks();
+			BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
+			List<AbstractSemanticAnalyzerHook> saHooks = getSemanticAnalyzerHooks();
 
-            // Do semantic analysis and plan generation
-            if (saHooks != null) {
-                HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
-                hookCtx.setConf(conf);
-                for (AbstractSemanticAnalyzerHook hook : saHooks) {
-                    tree = hook.preAnalyze(hookCtx, tree);
-                }
-                sem.analyze(tree, ctx);
-                for (AbstractSemanticAnalyzerHook hook : saHooks) {
-                    hook.postAnalyze(hookCtx, sem.getRootTasks());
-                }
-            } else {
-                sem.analyze(tree, ctx);
-            }
+			// Do semantic analysis and plan generation
+			if (saHooks != null) {
+				HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
+				hookCtx.setConf(conf);
+				for (AbstractSemanticAnalyzerHook hook : saHooks) {
+					tree = hook.preAnalyze(hookCtx, tree);
+				}
+				sem.analyze(tree, ctx);
+				for (AbstractSemanticAnalyzerHook hook : saHooks) {
+					hook.postAnalyze(hookCtx, sem.getRootTasks());
+				}
+			} else {
+				sem.analyze(tree, ctx);
+			}
 
-            LOG.info("Semantic Analysis Completed");
+			LOG.info("Semantic Analysis Completed");
 
-            // validate the plan
-            sem.validate();
+			// validate the plan
+			sem.validate();
 
-            plan = new QueryPlan(command, sem);
-            // initialize FetchTask right here
-            if (plan.getFetchTask() != null) {
-                plan.getFetchTask().initialize(conf, plan, null);
-            }
+			plan = new QueryPlan(command, sem);
+			// initialize FetchTask right here
+			if (plan.getFetchTask() != null) {
+				plan.getFetchTask().initialize(conf, plan, null);
+			}
 
-            // get the output schema
-            schema = getSchema(sem, conf);
+			// get the output schema
+			schema = getSchema(sem, conf);
 
-            // test Only - serialize the query plan and deserialize it
-            if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
+			// test Only - serialize the query plan and deserialize it
+			if (sem instanceof SemanticAnalyzer
+					&& command.toLowerCase().indexOf("create") < 0) {
 
-                Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+				Thread.currentThread().setContextClassLoader(
+						this.getClass().getClassLoader());
 
-                String queryPlanFileName = ctx.getLocalScratchDir(true) + Path.SEPARATOR_CHAR + "queryplan.xml";
-                LOG.info("query plan = " + queryPlanFileName);
-                queryPlanFileName = new Path(queryPlanFileName).toUri().getPath();
+				String queryPlanFileName = ctx.getLocalScratchDir(true)
+						+ Path.SEPARATOR_CHAR + "queryplan.xml";
+				LOG.info("query plan = " + queryPlanFileName);
+				queryPlanFileName = new Path(queryPlanFileName).toUri()
+						.getPath();
 
-                // serialize the queryPlan
-                FileOutputStream fos = new FileOutputStream(queryPlanFileName);
-                Utilities.serializeQueryPlan(plan, fos);
-                fos.close();
+				// serialize the queryPlan
+				FileOutputStream fos = new FileOutputStream(queryPlanFileName);
+				Utilities.serializeQueryPlan(plan, fos);
+				fos.close();
 
-                // deserialize the queryPlan
-                FileInputStream fis = new FileInputStream(queryPlanFileName);
-                QueryPlan newPlan = Utilities.deserializeQueryPlan(fis, conf);
-                fis.close();
+				// deserialize the queryPlan
+				FileInputStream fis = new FileInputStream(queryPlanFileName);
+				QueryPlan newPlan = Utilities.deserializeQueryPlan(fis, conf);
+				fis.close();
 
-                // Use the deserialized plan
-                plan = newPlan;
-            }
+				// Use the deserialized plan
+				plan = newPlan;
+			}
 
-            // initialize FetchTask right here
-            if (plan.getFetchTask() != null) {
-                plan.getFetchTask().initialize(conf, plan, null);
-            }
+			// initialize FetchTask right here
+			if (plan.getFetchTask() != null) {
+				plan.getFetchTask().initialize(conf, plan, null);
+			}
 
-            // do the authorization check
-            if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
-                try {
-                    // doAuthorization(sem);
-                } catch (AuthorizationException authExp) {
-                    console.printError("Authorization failed:" + authExp.getMessage()
-                            + ". Use show grant to get more details.");
-                    return 403;
-                }
-            }
+			// do the authorization check
+			if (HiveConf.getBoolVar(conf,
+					HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+				try {
+					// doAuthorization(sem);
+				} catch (AuthorizationException authExp) {
+					console.printError("Authorization failed:"
+							+ authExp.getMessage()
+							+ ". Use show grant to get more details.");
+					return 403;
+				}
+			}
 
-            // hyracks run
-            if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
-                hivesterix = true;
-                return engine.compileJob(sem.getRootTasks());
-            }
+			// hyracks run
+			if (sem instanceof SemanticAnalyzer
+					&& command.toLowerCase().indexOf("create") < 0) {
+				hivesterix = true;
+				return engine.compileJob(sem.getRootTasks());
+			}
 
-            return 0;
-        } catch (SemanticException e) {
-            errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
-            SQLState = ErrorMsg.findSQLState(e.getMessage());
-            console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (10);
-        } catch (ParseException e) {
-            errorMessage = "FAILED: Parse Error: " + e.getMessage();
-            SQLState = ErrorMsg.findSQLState(e.getMessage());
-            console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (11);
-        } catch (Exception e) {
-            errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
-            SQLState = ErrorMsg.findSQLState(e.getMessage());
-            console.printError(errorMessage + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (12);
-        }
-    }
+			return 0;
+		} catch (SemanticException e) {
+			errorMessage = "FAILED: Error in semantic analysis: "
+					+ e.getMessage();
+			SQLState = ErrorMsg.findSQLState(e.getMessage());
+			console.printError(errorMessage, "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (10);
+		} catch (ParseException e) {
+			errorMessage = "FAILED: Parse Error: " + e.getMessage();
+			SQLState = ErrorMsg.findSQLState(e.getMessage());
+			console.printError(errorMessage, "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (11);
+		} catch (Exception e) {
+			errorMessage = "FAILED: Hive Internal Error: "
+					+ Utilities.getNameMessage(e);
+			SQLState = ErrorMsg.findSQLState(e.getMessage());
+			console.printError(errorMessage + "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (12);
+		}
+	}
 
-    private void doAuthorization(BaseSemanticAnalyzer sem) throws HiveException, AuthorizationException {
-        HashSet<ReadEntity> inputs = sem.getInputs();
-        HashSet<WriteEntity> outputs = sem.getOutputs();
-        SessionState ss = SessionState.get();
-        HiveOperation op = ss.getHiveOperation();
-        Hive db = sem.getDb();
-        if (op != null) {
-            if (op.equals(HiveOperation.CREATETABLE_AS_SELECT) || op.equals(HiveOperation.CREATETABLE)) {
-                ss.getAuthorizer().authorize(db.getDatabase(db.getCurrentDatabase()), null,
-                        HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
-            } else {
-                // if (op.equals(HiveOperation.IMPORT)) {
-                // ImportSemanticAnalyzer isa = (ImportSemanticAnalyzer) sem;
-                // if (!isa.existsTable()) {
-                ss.getAuthorizer().authorize(db.getDatabase(db.getCurrentDatabase()), null,
-                        HiveOperation.CREATETABLE_AS_SELECT.getOutputRequiredPrivileges());
-                // }
-                // }
-            }
-            if (outputs != null && outputs.size() > 0) {
-                for (WriteEntity write : outputs) {
+	private void doAuthorization(BaseSemanticAnalyzer sem)
+			throws HiveException, AuthorizationException {
+		HashSet<ReadEntity> inputs = sem.getInputs();
+		HashSet<WriteEntity> outputs = sem.getOutputs();
+		SessionState ss = SessionState.get();
+		HiveOperation op = ss.getHiveOperation();
+		Hive db = sem.getDb();
+		if (op != null) {
+			if (op.equals(HiveOperation.CREATETABLE_AS_SELECT)
+					|| op.equals(HiveOperation.CREATETABLE)) {
+				ss.getAuthorizer().authorize(
+						db.getDatabase(db.getCurrentDatabase()),
+						null,
+						HiveOperation.CREATETABLE_AS_SELECT
+								.getOutputRequiredPrivileges());
+			} else {
+				// if (op.equals(HiveOperation.IMPORT)) {
+				// ImportSemanticAnalyzer isa = (ImportSemanticAnalyzer) sem;
+				// if (!isa.existsTable()) {
+				ss.getAuthorizer().authorize(
+						db.getDatabase(db.getCurrentDatabase()),
+						null,
+						HiveOperation.CREATETABLE_AS_SELECT
+								.getOutputRequiredPrivileges());
+				// }
+				// }
+			}
+			if (outputs != null && outputs.size() > 0) {
+				for (WriteEntity write : outputs) {
 
-                    if (write.getType() == WriteEntity.Type.PARTITION) {
-                        Partition part = db.getPartition(write.getTable(), write.getPartition().getSpec(), false);
-                        if (part != null) {
-                            ss.getAuthorizer().authorize(write.getPartition(), null, op.getOutputRequiredPrivileges());
-                            continue;
-                        }
-                    }
+					if (write.getType() == WriteEntity.Type.PARTITION) {
+						Partition part = db.getPartition(write.getTable(),
+								write.getPartition().getSpec(), false);
+						if (part != null) {
+							ss.getAuthorizer().authorize(write.getPartition(),
+									null, op.getOutputRequiredPrivileges());
+							continue;
+						}
+					}
 
-                    if (write.getTable() != null) {
-                        ss.getAuthorizer().authorize(write.getTable(), null, op.getOutputRequiredPrivileges());
-                    }
-                }
+					if (write.getTable() != null) {
+						ss.getAuthorizer().authorize(write.getTable(), null,
+								op.getOutputRequiredPrivileges());
+					}
+				}
 
-            }
-        }
+			}
+		}
 
-        if (inputs != null && inputs.size() > 0) {
+		if (inputs != null && inputs.size() > 0) {
 
-            Map<Table, List<String>> tab2Cols = new HashMap<Table, List<String>>();
-            Map<Partition, List<String>> part2Cols = new HashMap<Partition, List<String>>();
+			Map<Table, List<String>> tab2Cols = new HashMap<Table, List<String>>();
+			Map<Partition, List<String>> part2Cols = new HashMap<Partition, List<String>>();
 
-            Map<String, Boolean> tableUsePartLevelAuth = new HashMap<String, Boolean>();
-            for (ReadEntity read : inputs) {
-                if (read.getPartition() != null) {
-                    Table tbl = read.getTable();
-                    String tblName = tbl.getTableName();
-                    if (tableUsePartLevelAuth.get(tblName) == null) {
-                        boolean usePartLevelPriv = (tbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
-                                .equalsIgnoreCase(tbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))));
-                        if (usePartLevelPriv) {
-                            tableUsePartLevelAuth.put(tblName, Boolean.TRUE);
-                        } else {
-                            tableUsePartLevelAuth.put(tblName, Boolean.FALSE);
-                        }
-                    }
-                }
-            }
+			Map<String, Boolean> tableUsePartLevelAuth = new HashMap<String, Boolean>();
+			for (ReadEntity read : inputs) {
+				if (read.getPartition() != null) {
+					Table tbl = read.getTable();
+					String tblName = tbl.getTableName();
+					if (tableUsePartLevelAuth.get(tblName) == null) {
+						boolean usePartLevelPriv = (tbl.getParameters().get(
+								"PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
+								.equalsIgnoreCase(tbl.getParameters().get(
+										"PARTITION_LEVEL_PRIVILEGE"))));
+						if (usePartLevelPriv) {
+							tableUsePartLevelAuth.put(tblName, Boolean.TRUE);
+						} else {
+							tableUsePartLevelAuth.put(tblName, Boolean.FALSE);
+						}
+					}
+				}
+			}
 
-            if (op.equals(HiveOperation.CREATETABLE_AS_SELECT) || op.equals(HiveOperation.QUERY)) {
-                SemanticAnalyzer querySem = (SemanticAnalyzer) sem;
-                ParseContext parseCtx = querySem.getParseContext();
-                Map<TableScanOperator, Table> tsoTopMap = parseCtx.getTopToTable();
+			if (op.equals(HiveOperation.CREATETABLE_AS_SELECT)
+					|| op.equals(HiveOperation.QUERY)) {
+				SemanticAnalyzer querySem = (SemanticAnalyzer) sem;
+				ParseContext parseCtx = querySem.getParseContext();
+				Map<TableScanOperator, Table> tsoTopMap = parseCtx
+						.getTopToTable();
 
-                for (Map.Entry<String, Operator<? extends Serializable>> topOpMap : querySem.getParseContext()
-                        .getTopOps().entrySet()) {
-                    Operator<? extends Serializable> topOp = topOpMap.getValue();
-                    if (topOp instanceof TableScanOperator && tsoTopMap.containsKey(topOp)) {
-                        TableScanOperator tableScanOp = (TableScanOperator) topOp;
-                        Table tbl = tsoTopMap.get(tableScanOp);
-                        List<Integer> neededColumnIds = tableScanOp.getNeededColumnIDs();
-                        List<FieldSchema> columns = tbl.getCols();
-                        List<String> cols = new ArrayList<String>();
-                        if (neededColumnIds != null && neededColumnIds.size() > 0) {
-                            for (int i = 0; i < neededColumnIds.size(); i++) {
-                                cols.add(columns.get(neededColumnIds.get(i)).getName());
-                            }
-                        } else {
-                            for (int i = 0; i < columns.size(); i++) {
-                                cols.add(columns.get(i).getName());
-                            }
-                        }
-                        if (tbl.isPartitioned() && tableUsePartLevelAuth.get(tbl.getTableName())) {
-                            String alias_id = topOpMap.getKey();
-                            PrunedPartitionList partsList = PartitionPruner.prune(parseCtx.getTopToTable().get(topOp),
-                                    parseCtx.getOpToPartPruner().get(topOp), parseCtx.getConf(), alias_id,
-                                    parseCtx.getPrunedPartitions());
-                            Set<Partition> parts = new HashSet<Partition>();
-                            parts.addAll(partsList.getConfirmedPartns());
-                            parts.addAll(partsList.getUnknownPartns());
-                            for (Partition part : parts) {
-                                List<String> existingCols = part2Cols.get(part);
-                                if (existingCols == null) {
-                                    existingCols = new ArrayList<String>();
-                                }
-                                existingCols.addAll(cols);
-                                part2Cols.put(part, existingCols);
-                            }
-                        } else {
-                            List<String> existingCols = tab2Cols.get(tbl);
-                            if (existingCols == null) {
-                                existingCols = new ArrayList<String>();
-                            }
-                            existingCols.addAll(cols);
-                            tab2Cols.put(tbl, existingCols);
-                        }
-                    }
-                }
-            }
+				for (Map.Entry<String, Operator<? extends Serializable>> topOpMap : querySem
+						.getParseContext().getTopOps().entrySet()) {
+					Operator<? extends Serializable> topOp = topOpMap
+							.getValue();
+					if (topOp instanceof TableScanOperator
+							&& tsoTopMap.containsKey(topOp)) {
+						TableScanOperator tableScanOp = (TableScanOperator) topOp;
+						Table tbl = tsoTopMap.get(tableScanOp);
+						List<Integer> neededColumnIds = tableScanOp
+								.getNeededColumnIDs();
+						List<FieldSchema> columns = tbl.getCols();
+						List<String> cols = new ArrayList<String>();
+						if (neededColumnIds != null
+								&& neededColumnIds.size() > 0) {
+							for (int i = 0; i < neededColumnIds.size(); i++) {
+								cols.add(columns.get(neededColumnIds.get(i))
+										.getName());
+							}
+						} else {
+							for (int i = 0; i < columns.size(); i++) {
+								cols.add(columns.get(i).getName());
+							}
+						}
+						if (tbl.isPartitioned()
+								&& tableUsePartLevelAuth
+										.get(tbl.getTableName())) {
+							String alias_id = topOpMap.getKey();
+							PrunedPartitionList partsList = PartitionPruner
+									.prune(parseCtx.getTopToTable().get(topOp),
+											parseCtx.getOpToPartPruner().get(
+													topOp), parseCtx.getConf(),
+											alias_id,
+											parseCtx.getPrunedPartitions());
+							Set<Partition> parts = new HashSet<Partition>();
+							parts.addAll(partsList.getConfirmedPartns());
+							parts.addAll(partsList.getUnknownPartns());
+							for (Partition part : parts) {
+								List<String> existingCols = part2Cols.get(part);
+								if (existingCols == null) {
+									existingCols = new ArrayList<String>();
+								}
+								existingCols.addAll(cols);
+								part2Cols.put(part, existingCols);
+							}
+						} else {
+							List<String> existingCols = tab2Cols.get(tbl);
+							if (existingCols == null) {
+								existingCols = new ArrayList<String>();
+							}
+							existingCols.addAll(cols);
+							tab2Cols.put(tbl, existingCols);
+						}
+					}
+				}
+			}
 
-            // cache the results for table authorization
-            Set<String> tableAuthChecked = new HashSet<String>();
-            for (ReadEntity read : inputs) {
-                Table tbl = null;
-                if (read.getPartition() != null) {
-                    tbl = read.getPartition().getTable();
-                    // use partition level authorization
-                    if (tableUsePartLevelAuth.get(tbl.getTableName())) {
-                        List<String> cols = part2Cols.get(read.getPartition());
-                        if (cols != null && cols.size() > 0) {
-                            ss.getAuthorizer().authorize(read.getPartition().getTable(), read.getPartition(), cols,
-                                    op.getInputRequiredPrivileges(), null);
-                        } else {
-                            ss.getAuthorizer().authorize(read.getPartition(), op.getInputRequiredPrivileges(), null);
-                        }
-                        continue;
-                    }
-                } else if (read.getTable() != null) {
-                    tbl = read.getTable();
-                }
+			// cache the results for table authorization
+			Set<String> tableAuthChecked = new HashSet<String>();
+			for (ReadEntity read : inputs) {
+				Table tbl = null;
+				if (read.getPartition() != null) {
+					tbl = read.getPartition().getTable();
+					// use partition level authorization
+					if (tableUsePartLevelAuth.get(tbl.getTableName())) {
+						List<String> cols = part2Cols.get(read.getPartition());
+						if (cols != null && cols.size() > 0) {
+							ss.getAuthorizer().authorize(
+									read.getPartition().getTable(),
+									read.getPartition(), cols,
+									op.getInputRequiredPrivileges(), null);
+						} else {
+							ss.getAuthorizer().authorize(read.getPartition(),
+									op.getInputRequiredPrivileges(), null);
+						}
+						continue;
+					}
+				} else if (read.getTable() != null) {
+					tbl = read.getTable();
+				}
 
-                // if we reach here, it means it needs to do a table
-                // authorization
-                // check, and the table authorization may already happened
-                // because of other
-                // partitions
-                if (tbl != null && !tableAuthChecked.contains(tbl.getTableName())) {
-                    List<String> cols = tab2Cols.get(tbl);
-                    if (cols != null && cols.size() > 0) {
-                        ss.getAuthorizer().authorize(tbl, null, cols, op.getInputRequiredPrivileges(), null);
-                    } else {
-                        ss.getAuthorizer().authorize(tbl, op.getInputRequiredPrivileges(), null);
-                    }
-                    tableAuthChecked.add(tbl.getTableName());
-                }
-            }
+				// if we reach here, it means it needs to do a table
+				// authorization
+				// check, and the table authorization may already happened
+				// because of other
+				// partitions
+				if (tbl != null
+						&& !tableAuthChecked.contains(tbl.getTableName())) {
+					List<String> cols = tab2Cols.get(tbl);
+					if (cols != null && cols.size() > 0) {
+						ss.getAuthorizer().authorize(tbl, null, cols,
+								op.getInputRequiredPrivileges(), null);
+					} else {
+						ss.getAuthorizer().authorize(tbl,
+								op.getInputRequiredPrivileges(), null);
+					}
+					tableAuthChecked.add(tbl.getTableName());
+				}
+			}
 
-        }
-    }
+		}
+	}
 
-    /**
-     * @return The current query plan associated with this Driver, if any.
-     */
-    public QueryPlan getPlan() {
-        return plan;
-    }
+	/**
+	 * @return The current query plan associated with this Driver, if any.
+	 */
+	public QueryPlan getPlan() {
+		return plan;
+	}
 
-    /**
-     * @param t
-     *            The table to be locked
-     * @param p
-     *            The partition to be locked
-     * @param mode
-     *            The mode of the lock (SHARED/EXCLUSIVE) Get the list of
-     *            objects to be locked. If a partition needs to be locked (in
-     *            any mode), all its parents should also be locked in SHARED
-     *            mode.
-     **/
-    private List<HiveLockObj> getLockObjects(Table t, Partition p, HiveLockMode mode) throws SemanticException {
-        List<HiveLockObj> locks = new LinkedList<HiveLockObj>();
+	/**
+	 * @param t
+	 *            The table to be locked
+	 * @param p
+	 *            The partition to be locked
+	 * @param mode
+	 *            The mode of the lock (SHARED/EXCLUSIVE) Get the list of
+	 *            objects to be locked. If a partition needs to be locked (in
+	 *            any mode), all its parents should also be locked in SHARED
+	 *            mode.
+	 **/
+	private List<HiveLockObj> getLockObjects(Table t, Partition p,
+			HiveLockMode mode) throws SemanticException {
+		List<HiveLockObj> locks = new LinkedList<HiveLockObj>();
 
-        HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(), String.valueOf(System
-                .currentTimeMillis()), "IMPLICIT");
+		HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(),
+				String.valueOf(System.currentTimeMillis()), "IMPLICIT");
 
-        if (t != null) {
-            locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode));
-            mode = HiveLockMode.SHARED;
-            locks.add(new HiveLockObj(new HiveLockObject(t.getDbName(), lockData), mode));
-            return locks;
-        }
+		if (t != null) {
+			locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode));
+			mode = HiveLockMode.SHARED;
+			locks.add(new HiveLockObj(new HiveLockObject(t.getDbName(),
+					lockData), mode));
+			return locks;
+		}
 
-        if (p != null) {
-            if (!(p instanceof DummyPartition)) {
-                locks.add(new HiveLockObj(new HiveLockObject(p, lockData), mode));
-            }
+		if (p != null) {
+			if (!(p instanceof DummyPartition)) {
+				locks.add(new HiveLockObj(new HiveLockObject(p, lockData), mode));
+			}
 
-            // All the parents are locked in shared mode
-            mode = HiveLockMode.SHARED;
+			// All the parents are locked in shared mode
+			mode = HiveLockMode.SHARED;
 
-            // For dummy partitions, only partition name is needed
-            String name = p.getName();
+			// For dummy partitions, only partition name is needed
+			String name = p.getName();
 
-            if (p instanceof DummyPartition) {
-                name = p.getName().split("@")[2];
-            }
+			if (p instanceof DummyPartition) {
+				name = p.getName().split("@")[2];
+			}
 
-            String partName = name;
-            String partialName = "";
-            String[] partns = name.split("/");
-            int len = p instanceof DummyPartition ? partns.length : partns.length - 1;
-            for (int idx = 0; idx < len; idx++) {
-                String partn = partns[idx];
-                partialName += partn;
-                try {
-                    locks.add(new HiveLockObj(new HiveLockObject(new DummyPartition(p.getTable(), p.getTable()
-                            .getDbName() + "/" + p.getTable().getTableName() + "/" + partialName), lockData), mode));
-                    partialName += "/";
-                } catch (HiveException e) {
-                    throw new SemanticException(e.getMessage());
-                }
-            }
+			String partName = name;
+			String partialName = "";
+			String[] partns = name.split("/");
+			int len = p instanceof DummyPartition ? partns.length
+					: partns.length - 1;
+			for (int idx = 0; idx < len; idx++) {
+				String partn = partns[idx];
+				partialName += partn;
+				try {
+					locks.add(new HiveLockObj(new HiveLockObject(
+							new DummyPartition(p.getTable(), p.getTable()
+									.getDbName()
+									+ "/"
+									+ p.getTable().getTableName()
+									+ "/"
+									+ partialName), lockData), mode));
+					partialName += "/";
+				} catch (HiveException e) {
+					throw new SemanticException(e.getMessage());
+				}
+			}
 
-            locks.add(new HiveLockObj(new HiveLockObject(p.getTable(), lockData), mode));
-            locks.add(new HiveLockObj(new HiveLockObject(p.getTable().getDbName(), lockData), mode));
-        }
-        return locks;
-    }
+			locks.add(new HiveLockObj(
+					new HiveLockObject(p.getTable(), lockData), mode));
+			locks.add(new HiveLockObj(new HiveLockObject(p.getTable()
+					.getDbName(), lockData), mode));
+		}
+		return locks;
+	}
 
-    /**
-     * Acquire read and write locks needed by the statement. The list of objects
-     * to be locked are obtained from he inputs and outputs populated by the
-     * compiler. The lock acuisition scheme is pretty simple. If all the locks
-     * cannot be obtained, error out. Deadlock is avoided by making sure that
-     * the locks are lexicographically sorted.
-     **/
-    public int acquireReadWriteLocks() {
-        try {
-            int sleepTime = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
-            int numRetries = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
+	/**
+	 * Acquire read and write locks needed by the statement. The list of objects
+	 * to be locked are obtained from he inputs and outputs populated by the
+	 * compiler. The lock acuisition scheme is pretty simple. If all the locks
+	 * cannot be obtained, error out. Deadlock is avoided by making sure that
+	 * the locks are lexicographically sorted.
+	 **/
+	public int acquireReadWriteLocks() {
+		try {
+			int sleepTime = conf
+					.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
+			int numRetries = conf
+					.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
 
-            boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
-            if (!supportConcurrency) {
-                return 0;
-            }
+			boolean supportConcurrency = conf
+					.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
+			if (!supportConcurrency) {
+				return 0;
+			}
 
-            List<HiveLockObj> lockObjects = new ArrayList<HiveLockObj>();
+			List<HiveLockObj> lockObjects = new ArrayList<HiveLockObj>();
 
-            // Sort all the inputs, outputs.
-            // If a lock needs to be acquired on any partition, a read lock
-            // needs to be acquired on all
-            // its parents also
-            for (ReadEntity input : plan.getInputs()) {
-                if (input.getType() == ReadEntity.Type.TABLE) {
-                    lockObjects.addAll(getLockObjects(input.getTable(), null, HiveLockMode.SHARED));
-                } else {
-                    lockObjects.addAll(getLockObjects(null, input.getPartition(), HiveLockMode.SHARED));
-                }
-            }
+			// Sort all the inputs, outputs.
+			// If a lock needs to be acquired on any partition, a read lock
+			// needs to be acquired on all
+			// its parents also
+			for (ReadEntity input : plan.getInputs()) {
+				if (input.getType() == ReadEntity.Type.TABLE) {
+					lockObjects.addAll(getLockObjects(input.getTable(), null,
+							HiveLockMode.SHARED));
+				} else {
+					lockObjects.addAll(getLockObjects(null,
+							input.getPartition(), HiveLockMode.SHARED));
+				}
+			}
 
-            for (WriteEntity output : plan.getOutputs()) {
-                if (output.getTyp() == WriteEntity.Type.TABLE) {
-                    lockObjects.addAll(getLockObjects(output.getTable(), null,
-                            output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED));
-                } else if (output.getTyp() == WriteEntity.Type.PARTITION) {
-                    lockObjects.addAll(getLockObjects(null, output.getPartition(), HiveLockMode.EXCLUSIVE));
-                }
-                // In case of dynamic queries, it is possible to have incomplete
-                // dummy partitions
-                else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) {
-                    lockObjects.addAll(getLockObjects(null, output.getPartition(), HiveLockMode.SHARED));
-                }
-            }
+			for (WriteEntity output : plan.getOutputs()) {
+				if (output.getTyp() == WriteEntity.Type.TABLE) {
+					lockObjects.addAll(getLockObjects(output.getTable(), null,
+							output.isComplete() ? HiveLockMode.EXCLUSIVE
+									: HiveLockMode.SHARED));
+				} else if (output.getTyp() == WriteEntity.Type.PARTITION) {
+					lockObjects.addAll(getLockObjects(null,
+							output.getPartition(), HiveLockMode.EXCLUSIVE));
+				}
+				// In case of dynamic queries, it is possible to have incomplete
+				// dummy partitions
+				else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) {
+					lockObjects.addAll(getLockObjects(null,
+							output.getPartition(), HiveLockMode.SHARED));
+				}
+			}
 
-            if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) {
-                return 0;
-            }
+			if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) {
+				return 0;
+			}
 
-            int ret = checkLockManager();
-            if (ret != 0) {
-                return ret;
-            }
+			int ret = checkLockManager();
+			if (ret != 0) {
+				return ret;
+			}
 
-            HiveLockObjectData lockData = new HiveLockObjectData(plan.getQueryId(), String.valueOf(System
-                    .currentTimeMillis()), "IMPLICIT");
+			HiveLockObjectData lockData = new HiveLockObjectData(
+					plan.getQueryId(), String.valueOf(System
+							.currentTimeMillis()), "IMPLICIT");
 
-            // Lock the database also
-            try {
-                Hive db = Hive.get(conf);
-                lockObjects.add(new HiveLockObj(new HiveLockObject(db.getCurrentDatabase(), lockData),
-                        HiveLockMode.SHARED));
-            } catch (HiveException e) {
-                throw new SemanticException(e.getMessage());
-            }
+			// Lock the database also
+			try {
+				Hive db = Hive.get(conf);
+				lockObjects.add(new HiveLockObj(new HiveLockObject(db
+						.getCurrentDatabase(), lockData), HiveLockMode.SHARED));
+			} catch (HiveException e) {
+				throw new SemanticException(e.getMessage());
+			}
 
-            ctx.setHiveLockMgr(hiveLockMgr);
-            List<HiveLock> hiveLocks = null;
+			ctx.setHiveLockMgr(hiveLockMgr);
+			List<HiveLock> hiveLocks = null;
 
-            int tryNum = 1;
-            do {
+			int tryNum = 1;
+			do {
 
-                // ctx.getHiveLockMgr();
-                // hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false);
+				// ctx.getHiveLockMgr();
+				// hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false);
 
-                if (hiveLocks != null) {
-                    break;
-                }
+				if (hiveLocks != null) {
+					break;
+				}
 
-                tryNum++;
-                try {
-                    Thread.sleep(sleepTime);
-                } catch (InterruptedException e) {
-                }
-            } while (tryNum < numRetries);
+				tryNum++;
+				try {
+					Thread.sleep(sleepTime);
+				} catch (InterruptedException e) {
+				}
+			} while (tryNum < numRetries);
 
-            if (hiveLocks == null) {
-                throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg());
-            } else {
-                ctx.setHiveLocks(hiveLocks);
-            }
+			if (hiveLocks == null) {
+				throw new SemanticException(
+						ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg());
+			} else {
+				ctx.setHiveLocks(hiveLocks);
+			}
 
-            return (0);
-        } catch (SemanticException e) {
-            errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage();
-            SQLState = ErrorMsg.findSQLState(e.getMessage());
-            console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (10);
-        } catch (Exception e) {
-            errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage();
-            SQLState = ErrorMsg.findSQLState(e.getMessage());
-            console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (10);
-        }
-    }
+			return (0);
+		} catch (SemanticException e) {
+			errorMessage = "FAILED: Error in acquiring locks: "
+					+ e.getMessage();
+			SQLState = ErrorMsg.findSQLState(e.getMessage());
+			console.printError(errorMessage, "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (10);
+		} catch (Exception e) {
+			errorMessage = "FAILED: Error in acquiring locks: "
+					+ e.getMessage();
+			SQLState = ErrorMsg.findSQLState(e.getMessage());
+			console.printError(errorMessage, "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (10);
+		}
+	}
 
-    /**
-     * Release all the locks acquired implicitly by the statement. Note that the
-     * locks acquired with 'keepAlive' set to True are not released.
-     **/
-    private void releaseLocks() {
-        if (ctx != null && ctx.getHiveLockMgr() != null) {
-            try {
-                ctx.getHiveLockMgr().close();
-                ctx.setHiveLocks(null);
-            } catch (LockException e) {
-            }
-        }
-    }
+	/**
+	 * Release all the locks acquired implicitly by the statement. Note that the
+	 * locks acquired with 'keepAlive' set to True are not released.
+	 **/
+	private void releaseLocks() {
+		if (ctx != null && ctx.getHiveLockMgr() != null) {
+			try {
+				ctx.getHiveLockMgr().close();
+				ctx.setHiveLocks(null);
+			} catch (LockException e) {
+			}
+		}
+	}
 
-    /**
-     * @param hiveLocks
-     *            list of hive locks to be released Release all the locks
-     *            specified. If some of the locks have already been released,
-     *            ignore them
-     **/
-    private void releaseLocks(List<HiveLock> hiveLocks) {
-        if (hiveLocks != null) {
-            ctx.getHiveLockMgr().releaseLocks(hiveLocks);
-        }
-        ctx.setHiveLocks(null);
-    }
+	/**
+	 * @param hiveLocks
+	 *            list of hive locks to be released Release all the locks
+	 *            specified. If some of the locks have already been released,
+	 *            ignore them
+	 **/
+	private void releaseLocks(List<HiveLock> hiveLocks) {
+		if (hiveLocks != null) {
+			ctx.getHiveLockMgr().releaseLocks(hiveLocks);
+		}
+		ctx.setHiveLocks(null);
+	}
 
-    public CommandProcessorResponse run(String command) {
-        errorMessage = null;
-        SQLState = null;
+	public CommandProcessorResponse run(String command) {
+		errorMessage = null;
+		SQLState = null;
 
-        int ret = compile(command);
-        if (ret != 0) {
-            // releaseLocks(ctx.getHiveLocks());
-            return new CommandProcessorResponse(ret, errorMessage, SQLState);
-        }
+		int ret = compile(command);
+		if (ret != 0) {
+			// releaseLocks(ctx.getHiveLocks());
+			return new CommandProcessorResponse(ret, errorMessage, SQLState);
+		}
 
-        // ret = acquireReadWriteLocks();
-        if (ret != 0) {
-            // releaseLocks(ctx.getHiveLocks());
-            return new CommandProcessorResponse(ret, errorMessage, SQLState);
-        }
+		// ret = acquireReadWriteLocks();
+		if (ret != 0) {
+			// releaseLocks(ctx.getHiveLocks());
+			return new CommandProcessorResponse(ret, errorMessage, SQLState);
+		}
 
-        ret = execute();
-        if (ret != 0) {
-            // releaseLocks(ctx.getHiveLocks());
-            return new CommandProcessorResponse(ret, errorMessage, SQLState);
-        }
+		ret = execute();
+		if (ret != 0) {
+			// releaseLocks(ctx.getHiveLocks());
+			return new CommandProcessorResponse(ret, errorMessage, SQLState);
+		}
 
-        // releaseLocks(ctx.getHiveLocks());
-        return new CommandProcessorResponse(ret);
-    }
+		// releaseLocks(ctx.getHiveLocks());
+		return new CommandProcessorResponse(ret);
+	}
 
-    private List<AbstractSemanticAnalyzerHook> getSemanticAnalyzerHooks() throws Exception {
-        ArrayList<AbstractSemanticAnalyzerHook> saHooks = new ArrayList<AbstractSemanticAnalyzerHook>();
-        String pestr = conf.getVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK);
-        if (pestr == null) {
-            return saHooks;
-        }
-        pestr = pestr.trim();
-        if (pestr.equals("")) {
-            return saHooks;
-        }
+	private List<AbstractSemanticAnalyzerHook> getSemanticAnalyzerHooks()
+			throws Exception {
+		ArrayList<AbstractSemanticAnalyzerHook> saHooks = new ArrayList<AbstractSemanticAnalyzerHook>();
+		String pestr = conf.getVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK);
+		if (pestr == null) {
+			return saHooks;
+		}
+		pestr = pestr.trim();
+		if (pestr.equals("")) {
+			return saHooks;
+		}
 
-        String[] peClasses = pestr.split(",");
+		String[] peClasses = pestr.split(",");
 
-        for (String peClass : peClasses) {
-            try {
-                AbstractSemanticAnalyzerHook hook = HiveUtils.getSemanticAnalyzerHook(conf, peClass);
-                saHooks.add(hook);
-            } catch (HiveException e) {
-                console.printError("Pre Exec Hook Class not found:" + e.getMessage());
-                throw e;
-            }
-        }
+		for (String peClass : peClasses) {
+			try {
+				AbstractSemanticAnalyzerHook hook = HiveUtils
+						.getSemanticAnalyzerHook(conf, peClass);
+				saHooks.add(hook);
+			} catch (HiveException e) {
+				console.printError("Pre Exec Hook Class not found:"
+						+ e.getMessage());
+				throw e;
+			}
+		}
 
-        return saHooks;
-    }
+		return saHooks;
+	}
 
-    private List<Hook> getPreExecHooks() throws Exception {
-        ArrayList<Hook> pehooks = new ArrayList<Hook>();
-        String pestr = conf.getVar(HiveConf.ConfVars.PREEXECHOOKS);
-        pestr = pestr.trim();
-        if (pestr.equals("")) {
-            return pehooks;
-        }
+	private List<Hook> getPreExecHooks() throws Exception {
+		ArrayList<Hook> pehooks = new ArrayList<Hook>();
+		String pestr = conf.getVar(HiveConf.ConfVars.PREEXECHOOKS);
+		pestr = pestr.trim();
+		if (pestr.equals("")) {
+			return pehooks;
+		}
 
-        String[] peClasses = pestr.split(",");
+		String[] peClasses = pestr.split(",");
 
-        for (String peClass : peClasses) {
-            try {
-                pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader()).newInstance());
-            } catch (ClassNotFoundException e) {
-                console.printError("Pre Exec Hook Class not found:" + e.getMessage());
-                throw e;
-            }
-        }
+		for (String peClass : peClasses) {
+			try {
+				pehooks.add((Hook) Class.forName(peClass.trim(), true,
+						JavaUtils.getClassLoader()).newInstance());
+			} catch (ClassNotFoundException e) {
+				console.printError("Pre Exec Hook Class not found:"
+						+ e.getMessage());
+				throw e;
+			}
+		}
 
-        return pehooks;
-    }
+		return pehooks;
+	}
 
-    private List<Hook> getPostExecHooks() throws Exception {
-        ArrayList<Hook> pehooks = new ArrayList<Hook>();
-        String pestr = conf.getVar(HiveConf.ConfVars.POSTEXECHOOKS);
-        pestr = pestr.trim();
-        if (pestr.equals("")) {
-            return pehooks;
-        }
+	private List<Hook> getPostExecHooks() throws Exception {
+		ArrayList<Hook> pehooks = new ArrayList<Hook>();
+		String pestr = conf.getVar(HiveConf.ConfVars.POSTEXECHOOKS);
+		pestr = pestr.trim();
+		if (pestr.equals("")) {
+			return pehooks;
+		}
 
-        String[] peClasses = pestr.split(",");
+		String[] peClasses = pestr.split(",");
 
-        for (String peClass : peClasses) {
-            try {
-                pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader()).newInstance());
-            } catch (ClassNotFoundException e) {
-                console.printError("Post Exec Hook Class not found:" + e.getMessage());
-                throw e;
-            }
-        }
+		for (String peClass : peClasses) {
+			try {
+				pehooks.add((Hook) Class.forName(peClass.trim(), true,
+						JavaUtils.getClassLoader()).newInstance());
+			} catch (ClassNotFoundException e) {
+				console.printError("Post Exec Hook Class not found:"
+						+ e.getMessage());
+				throw e;
+			}
+		}
 
-        return pehooks;
-    }
+		return pehooks;
+	}
 
-    public int execute() {
-        // execute hivesterix plan
-        if (hivesterix) {
-            hivesterix = false;
-            int ret = engine.executeJob();
-            if (ret != 0)
-                return ret;
-        }
+	public int execute() {
+		// execute hivesterix plan
+		if (hivesterix) {
+			hivesterix = false;
+			int ret = engine.executeJob();
+			if (ret != 0)
+				return ret;
+		}
 
-        boolean noName = StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
-        int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
+		boolean noName = StringUtils.isEmpty(conf
+				.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
+		int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
 
-        String queryId = plan.getQueryId();
-        String queryStr = plan.getQueryStr();
+		String queryId = plan.getQueryId();
+		String queryStr = plan.getQueryStr();
 
-        conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
-        conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
-        maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
+		conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+		conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
+		maxthreads = HiveConf.getIntVar(conf,
+				HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
 
-        try {
-            LOG.info("Starting command: " + queryStr);
+		try {
+			LOG.info("Starting command: " + queryStr);
 
-            plan.setStarted();
+			plan.setStarted();
 
-            if (SessionState.get() != null) {
-                SessionState.get().getHiveHistory().startQuery(queryStr, conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
-                SessionState.get().getHiveHistory().logPlanProgress(plan);
-            }
-            resStream = null;
+			if (SessionState.get() != null) {
+				SessionState
+						.get()
+						.getHiveHistory()
+						.startQuery(queryStr,
+								conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
+				SessionState.get().getHiveHistory().logPlanProgress(plan);
+			}
+			resStream = null;
 
-            HookContext hookContext = new HookContext(plan, conf);
+			HookContext hookContext = new HookContext(plan, conf);
 
-            for (Hook peh : getPreExecHooks()) {
-                if (peh instanceof ExecuteWithHookContext) {
-                    ((ExecuteWithHookContext) peh).run(hookContext);
-                } else if (peh instanceof PreExecute) {
-                    ((PreExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(), ShimLoader
-                            .getHadoopShims().getUGIForConf(conf));
-                }
-            }
+			for (Hook peh : getPreExecHooks()) {
+				if (peh instanceof ExecuteWithHookContext) {
+					((ExecuteWithHookContext) peh).run(hookContext);
+				} else if (peh instanceof PreExecute) {
+					((PreExecute) peh).run(SessionState.get(),
+							plan.getInputs(), plan.getOutputs(), ShimLoader
+									.getHadoopShims().getUGIForConf(conf));
+				}
+			}
 
-            int jobs = Utilities.getMRTasks(plan.getRootTasks()).size();
-            if (jobs > 0) {
-                console.printInfo("Total MapReduce jobs = " + jobs);
-            }
-            if (SessionState.get() != null) {
-                SessionState.get().getHiveHistory()
-                        .setQueryProperty(queryId, Keys.QUERY_NUM_TASKS, String.valueOf(jobs));
-                SessionState.get().getHiveHistory().setIdToTableMap(plan.getIdToTableNameMap());
-            }
-            String jobname = Utilities.abbreviate(queryStr, maxlen - 6);
+			int jobs = Utilities.getMRTasks(plan.getRootTasks()).size();
+			if (jobs > 0) {
+				console.printInfo("Total MapReduce jobs = " + jobs);
+			}
+			if (SessionState.get() != null) {
+				SessionState
+						.get()
+						.getHiveHistory()
+						.setQueryProperty(queryId, Keys.QUERY_NUM_TASKS,
+								String.valueOf(jobs));
+				SessionState.get().getHiveHistory()
+						.setIdToTableMap(plan.getIdToTableNameMap());
+			}
+			String jobname = Utilities.abbreviate(queryStr, maxlen - 6);
 
-            // A runtime that launches runnable tasks as separate Threads
-            // through
-            // TaskRunners
-            // As soon as a task isRunnable, it is put in a queue
-            // At any time, at most maxthreads tasks can be running
-            // The main thread polls the TaskRunners to check if they have
-            // finished.
+			// A runtime that launches runnable tasks as separate Threads
+			// through
+			// TaskRunners
+			// As soon as a task isRunnable, it is put in a queue
+			// At any time, at most maxthreads tasks can be running
+			// The main thread polls the TaskRunners to check if they have
+			// finished.
 
-            Queue<Task<? extends Serializable>> runnable = new LinkedList<Task<? extends Serializable>>();
-            Map<TaskResult, TaskRunner> running = new HashMap<TaskResult, TaskRunner>();
+			Queue<Task<? extends Serializable>> runnable = new LinkedList<Task<? extends Serializable>>();
+			Map<TaskResult, TaskRunner> running = new HashMap<TaskResult, TaskRunner>();
 
-            DriverContext driverCxt = new DriverContext(runnable, ctx);
+			DriverContext driverCxt = new DriverContext(runnable, ctx);
 
-            // Add root Tasks to runnable
+			// Add root Tasks to runnable
 
-            for (Task<? extends Serializable> tsk : plan.getRootTasks()) {
-                driverCxt.addToRunnable(tsk);
-            }
+			for (Task<? extends Serializable> tsk : plan.getRootTasks()) {
+				driverCxt.addToRunnable(tsk);
+			}
 
-            // Loop while you either have tasks running, or tasks queued up
+			// Loop while you either have tasks running, or tasks queued up
 
-            while (running.size() != 0 || runnable.peek() != null) {
-                // Launch upto maxthreads tasks
-                while (runnable.peek() != null && running.size() < maxthreads) {
-                    Task<? extends Serializable> tsk = runnable.remove();
-                    console.printInfo("executing task " + tsk.getName());
-                    launchTask(tsk, queryId, noName, running, jobname, jobs, driverCxt);
-                }
+			while (running.size() != 0 || runnable.peek() != null) {
+				// Launch upto maxthreads tasks
+				while (runnable.peek() != null && running.size() < maxthreads) {
+					Task<? extends Serializable> tsk = runnable.remove();
+					console.printInfo("executing task " + tsk.getName());
+					launchTask(tsk, queryId, noName, running, jobname, jobs,
+							driverCxt);
+				}
 
-                // poll the Tasks to see which one completed
-                TaskResult tskRes = pollTasks(running.keySet());
-                TaskRunner tskRun = running.remove(tskRes);
-                Task<? extends Serializable> tsk = tskRun.getTask();
-                hookContext.addCompleteTask(tskRun);
+				// poll the Tasks to see which one completed
+				TaskResult tskRes = pollTasks(running.keySet());
+				TaskRunner tskRun = running.remove(tskRes);
+				Task<? extends Serializable> tsk = tskRun.getTask();
+				hookContext.addCompleteTask(tskRun);
 
-                int exitVal = tskRes.getExitVal();
-                if (exitVal != 0) {
-                    Task<? extends Serializable> backupTask = tsk.getAndInitBackupTask();
-                    if (backupTask != null) {
-                        errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
-                                + tsk.getClass().getName();
-                        console.printError(errorMessage);
+				int exitVal = tskRes.getExitVal();
+				if (exitVal != 0) {
+					Task<? extends Serializable> backupTask = tsk
+							.getAndInitBackupTask();
+					if (backupTask != null) {
+						errorMessage = "FAILED: Execution Error, return code "
+								+ exitVal + " from " + tsk.getClass().getName();
+						console.printError(errorMessage);
 
-                        errorMessage = "ATTEMPT: Execute BackupTask: " + backupTask.getClass().getName();
-                        console.printError(errorMessage);
+						errorMessage = "ATTEMPT: Execute BackupTask: "
+								+ backupTask.getClass().getName();
+						console.printError(errorMessage);
 
-                        // add backup task to runnable
-                        if (DriverContext.isLaunchable(backupTask)) {
-                            driverCxt.addToRunnable(backupTask);
-                        }
-                        continue;
+						// add backup task to runnable
+						if (DriverContext.isLaunchable(backupTask)) {
+							driverCxt.addToRunnable(backupTask);
+						}
+						continue;
 
-                    } else {
-                        // TODO: This error messaging is not very informative.
-                        // Fix that.
-                        errorMessage = "FAILED: Execution Error, return code " + exitVal + " from "
-                                + tsk.getClass().getName();
-                        SQLState = "08S01";
-                        console.printError(errorMessage);
-                        if (running.size() != 0) {
-                            taskCleanup();
-                        }
-                        // in case we decided to run everything in local mode,
-                        // restore the
-                        // the jobtracker setting to its initial value
-                        ctx.restoreOriginalTracker();
-                        return 9;
-                    }
-                }
+					} else {
+						// TODO: This error messaging is not very informative.
+						// Fix that.
+						errorMessage = "FAILED: Execution Error, return code "
+								+ exitVal + " from " + tsk.getClass().getName();
+						SQLState = "08S01";
+						console.printError(errorMessage);
+						if (running.size() != 0) {
+							taskCleanup();
+						}
+						// in case we decided to run everything in local mode,
+						// restore the
+						// the jobtracker setting to its initial value
+						ctx.restoreOriginalTracker();
+						return 9;
+					}
+				}
 
-                if (SessionState.get() != null) {
-                    SessionState.get().getHiveHistory()
-                            .setTaskProperty(queryId, tsk.getId(), Keys.TASK_RET_CODE, String.valueOf(exitVal));
-                    SessionState.get().getHiveHistory().endTask(queryId, tsk);
-                }
+				if (SessionState.get() != null) {
+					SessionState
+							.get()
+							.getHiveHistory()
+							.setTaskProperty(queryId, tsk.getId(),
+									Keys.TASK_RET_CODE, String.valueOf(exitVal));
+					SessionState.get().getHiveHistory().endTask(queryId, tsk);
+				}
 
-                if (tsk.getChildTasks() != null) {
-                    for (Task<? extends Serializable> child : tsk.getChildTasks()) {
-                        // hivesterix: don't check launchable condition
-                        // if (DriverContext.isLaunchable(child)) {
-                        driverCxt.addToRunnable(child);
-                        // }
-                    }
-                }
-            }
+				if (tsk.getChildTasks() != null) {
+					for (Task<? extends Serializable> child : tsk
+							.getChildTasks()) {
+						// hivesterix: don't check launchable condition
+						// if (DriverContext.isLaunchable(child)) {
+						driverCxt.addToRunnable(child);
+						// }
+					}
+				}
+			}
 
-            // in case we decided to run everything in local mode, restore the
-            // the jobtracker setting to its initial value
-            ctx.restoreOriginalTracker();
+			// in case we decided to run everything in local mode, restore the
+			// the jobtracker setting to its initial value
+			ctx.restoreOriginalTracker();
 
-            // remove incomplete outputs.
-            // Some incomplete outputs may be added at the beginning, for eg:
-            // for dynamic partitions.
-            // remove them
-            HashSet<WriteEntity> remOutputs = new HashSet<WriteEntity>();
-            for (WriteEntity output : plan.getOutputs()) {
-                if (!output.isComplete()) {
-                    remOutputs.add(output);
-                }
-            }
+			// remove incomplete outputs.
+			// Some incomplete outputs may be added at the beginning, for eg:
+			// for dynamic partitions.
+			// remove them
+			HashSet<WriteEntity> remOutputs = new HashSet<WriteEntity>();
+			for (WriteEntity output : plan.getOutputs()) {
+				if (!output.isComplete()) {
+					remOutputs.add(output);
+				}
+			}
 
-            for (WriteEntity output : remOutputs) {
-                plan.getOutputs().remove(output);
-            }
+			for (WriteEntity output : remOutputs) {
+				plan.getOutputs().remove(output);
+			}
 
-            // Get all the post execution hooks and execute them.
-            for (Hook peh : getPostExecHooks()) {
-                if (peh instanceof ExecuteWithHookContext) {
-                    ((ExecuteWithHookContext) peh).run(hookContext);
-                } else if (peh instanceof PostExecute) {
-                    ((PostExecute) peh)
-                            .run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
-                                    (SessionState.get() != null ? SessionState.get().getLineageState().getLineageInfo()
-                                            : null), ShimLoader.getHadoopShims().getUGIForConf(conf));
-                }
-            }
+			// Get all the post execution hooks and execute them.
+			for (Hook peh : getPostExecHooks()) {
+				if (peh instanceof ExecuteWithHookContext) {
+					((ExecuteWithHookContext) peh).run(hookContext);
+				} else if (peh instanceof PostExecute) {
+					((PostExecute) peh)
+							.run(SessionState.get(),
+									plan.getInputs(),
+									plan.getOutputs(),
+									(SessionState.get() != null ? SessionState
+											.get().getLineageState()
+											.getLineageInfo() : null),
+									ShimLoader.getHadoopShims().getUGIForConf(
+											conf));
+				}
+			}
 
-            if (SessionState.get() != null) {
-                SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(0));
-                SessionState.get().getHiveHistory().printRowCount(queryId);
-            }
-        } catch (Exception e) {
-            if (SessionState.get() != null) {
-                SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(12));
-            }
-            // TODO: do better with handling types of Exception here
-            errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
-            SQLState = "08S01";
-            console.printError(errorMessage + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return (12);
-        } finally {
-            if (SessionState.get() != null) {
-                SessionState.get().getHiveHistory().endQuery(queryId);
-            }
-            if (noName) {
-                conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
-            }
-        }
-        plan.setDone();
+			if (SessionState.get() != null) {
+				SessionState
+						.get()
+						.getHiveHistory()
+						.setQueryProperty(queryId, Keys.QUERY_RET_CODE,
+								String.valueOf(0));
+				SessionState.get().getHiveHistory().printRowCount(queryId);
+			}
+		} catch (Exception e) {
+			if (SessionState.get() != null) {
+				SessionState
+						.get()
+						.getHiveHistory()
+						.setQueryProperty(queryId, Keys.QUERY_RET_CODE,
+								String.valueOf(12));
+			}
+			// TODO: do better with handling types of Exception here
+			errorMessage = "FAILED: Hive Internal Error: "
+					+ Utilities.getNameMessage(e);
+			SQLState = "08S01";
+			console.printError(errorMessage + "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return (12);
+		} finally {
+			if (SessionState.get() != null) {
+				SessionState.get().getHiveHistory().endQuery(queryId);
+			}
+			if (noName) {
+				conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
+			}
+		}
+		plan.setDone();
 
-        if (SessionState.get() != null) {
-            try {
-                SessionState.get().getHiveHistory().logPlanProgress(plan);
-            } catch (Exception e) {
-            }
-        }
-        console.printInfo("OK");
+		if (SessionState.get() != null) {
+			try {
+				SessionState.get().getHiveHistory().logPlanProgress(plan);
+			} catch (Exception e) {
+			}
+		}
+		console.printInfo("OK");
 
-        return (0);
-    }
+		return (0);
+	}
 
-    /**
-     * Launches a new task
-     * 
-     * @param tsk
-     *            task being launched
-     * @param queryId
-     *            Id of the query containing the task
-     * @param noName
-     *            whether the task has a name set
-     * @param running
-     *            map from taskresults to taskrunners
-     * @param jobname
-     *            name of the task, if it is a map-reduce job
-     * @param jobs
-     *            number of map-reduce jobs
-     * @param curJobNo
-     *            the sequential number of the next map-reduce job
-     * @return the updated number of last the map-reduce job launched
-     */
+	/**
+	 * Launches a new task
+	 * 
+	 * @param tsk
+	 *            task being launched
+	 * @param queryId
+	 *            Id of the query containing the task
+	 * @param noName
+	 *            whether the task has a name set
+	 * @param running
+	 *            map from taskresults to taskrunners
+	 * @param jobname
+	 *            name of the task, if it is a map-reduce job
+	 * @param jobs
+	 *            number of map-reduce jobs
+	 * @param curJobNo
+	 *            the sequential number of the next map-reduce job
+	 * @return the updated number of last the map-reduce job launched
+	 */
 
-    public void launchTask(Task<? extends Serializable> tsk, String queryId, boolean noName,
-            Map<TaskResult, TaskRunner> running, String jobname, int jobs, DriverContext cxt) {
+	public void launchTask(Task<? extends Serializable> tsk, String queryId,
+			boolean noName, Map<TaskResult, TaskRunner> running,
+			String jobname, int jobs, DriverContext cxt) {
 
-        if (SessionState.get() != null) {
-            SessionState.get().getHiveHistory().startTask(queryId, tsk, tsk.getClass().getName());
-        }
-        if (tsk.isMapRedTask() && !(tsk instanceof ConditionalTask)) {
-            if (noName) {
-                conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname + "(" + tsk.getId() + ")");
-            }
-            cxt.incCurJobNo(1);
-            console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of " + jobs);
-        }
-        tsk.initialize(conf, plan, cxt);
-        TaskResult tskRes = new TaskResult();
-        TaskRunner tskRun = new TaskRunner(tsk, tskRes);
+		if (SessionState.get() != null) {
+			SessionState.get().getHiveHistory()
+					.startTask(queryId, tsk, tsk.getClass().getName());
+		}
+		if (tsk.isMapRedTask() && !(tsk instanceof ConditionalTask)) {
+			if (noName) {
+				conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname + "("
+						+ tsk.getId() + ")");
+			}
+			cxt.incCurJobNo(1);
+			console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of "
+					+ jobs);
+		}
+		tsk.initialize(conf, plan, cxt);
+		TaskResult tskRes = new TaskResult();
+		TaskRunner tskRun = new TaskRunner(tsk, tskRes);
 
-        // HiveConf.getBoolVar(conf, HiveConf.ConfVars.EXECPARALLEL) &&
-        // Launch Task: hivesterix tweak
-        if (tsk instanceof MapRedTask || tsk instanceof StatsTask) {
-            // Launch it in the parallel mode, as a separate thread only for MR
-            // tasks
-            tskRes.setRunning(false);
-            tskRes.setExitVal(0);
-        } else if (tsk instanceof ConditionalTask) {
-            ConditionalTask condTask = (ConditionalTask) tsk;
-            ConditionalResolver crs = condTask.getResolver();
-            if (crs instanceof ConditionalResolverMergeFiles) {
-                tskRes.setRunning(false);
-                tskRes.setExitVal(0);
+		// HiveConf.getBoolVar(conf, HiveConf.ConfVars.EXECPARALLEL) &&
+		// Launch Task: hivesterix tweak
+		if (tsk instanceof MapRedTask || tsk instanceof StatsTask) {
+			// Launch it in the parallel mode, as a separate thread only for MR
+			// tasks
+			tskRes.setRunning(false);
+			tskRes.setExitVal(0);
+		} else if (tsk instanceof ConditionalTask) {
+			ConditionalTask condTask = (ConditionalTask) tsk;
+			ConditionalResolver crs = condTask.getResolver();
+			if (crs instanceof ConditionalResolverMergeFiles) {
+				tskRes.setRunning(false);
+				tskRes.setExitVal(0);
 
-                List<Task<? extends Serializable>> children = condTask.getListTasks();
-                for (Task<? extends Serializable> child : children)
-                    if (child instanceof MapRedTask)
-                        cxt.addToRunnable(child);
-            }
-        } else {
-            tskRun.runSequential();
-        }
-        running.put(tskRes, tskRun);
-        return;
-    }
+				List<Task<? extends Serializable>> children = condTask
+						.getListTasks();
+				for (Task<? extends Serializable> child : children)
+					if (child instanceof MapRedTask)
+						cxt.addToRunnable(child);
+			}
+		} else {
+			tskRun.runSequential();
+		}
+		running.put(tskRes, tskRun);
+		return;
+	}
 
-    /**
-     * Cleans up remaining tasks in case of failure
-     */
+	/**
+	 * Cleans up remaining tasks in case of failure
+	 */
 
-    public void taskCleanup() {
-        // The currently existing Shutdown hooks will be automatically called,
-        // killing the map-reduce processes.
-        // The non MR processes will be killed as well.
-        System.exit(9);
-    }
+	public void taskCleanup() {
+		// The currently existing Shutdown hooks will be automatically called,
+		// killing the map-reduce processes.
+		// The non MR processes will be killed as well.
+		System.exit(9);
+	}
 
-    /**
-     * Polls running tasks to see if a task has ended.
-     * 
-     * @param results
-     *            Set of result objects for running tasks
-     * @return The result object for any completed/failed task
-     */
+	/**
+	 * Polls running tasks to see if a task has ended.
+	 * 
+	 * @param results
+	 *            Set of result objects for running tasks
+	 * @return The result object for any completed/failed task
+	 */
 
-    public TaskResult pollTasks(Set<TaskResult> results) {
-        Iterator<TaskResult> resultIterator = results.iterator();
-        while (true) {
-            while (resultIterator.hasNext()) {
-                TaskResult tskRes = resultIterator.next();
-                if (tskRes.isRunning() == false) {
-                    return tskRes;
-                }
-            }
+	public TaskResult pollTasks(Set<TaskResult> results) {
+		Iterator<TaskResult> resultIterator = results.iterator();
+		while (true) {
+			while (resultIterator.hasNext()) {
+				TaskResult tskRes = resultIterator.next();
+				if (tskRes.isRunning() == false) {
+					return tskRes;
+				}
+			}
 
-            // In this loop, nothing was found
-            // Sleep 10 seconds and restart
-            try {
-                Thread.sleep(sleeptime);
-            } catch (InterruptedException ie) {
-                // Do Nothing
-                ;
-            }
-            resultIterator = results.iterator();
-        }
-    }
+			// In this loop, nothing was found
+			// Sleep 10 seconds and restart
+			try {
+				Thread.sleep(sleeptime);
+			} catch (InterruptedException ie) {
+				// Do Nothing
+				;
+			}
+			resultIterator = results.iterator();
+		}
+	}
 
-    public boolean getResults(ArrayList<String> res) throws IOException {
-        if (plan != null && plan.getFetchTask() != null) {
-            FetchTask ft = plan.getFetchTask();
-            ft.setMaxRows(maxRows);
-            return ft.fetch(res);
-        }
+	public boolean getResults(ArrayList<String> res) throws IOException {
+		if (plan != null && plan.getFetchTask() != null) {
+			FetchTask ft = plan.getFetchTask();
+			ft.setMaxRows(maxRows);
+			return ft.fetch(res);
+		}
 
-        if (resStream == null) {
-            resStream = ctx.getStream();
-        }
-        if (resStream == null) {
-            return false;
-        }
+		if (resStream == null) {
+			resStream = ctx.getStream();
+		}
+		if (resStream == null) {
+			return false;
+		}
 
-        int numRows = 0;
-        String row = null;
+		int numRows = 0;
+		String row = null;
 
-        while (numRows < maxRows) {
-            if (resStream == null) {
-                if (numRows > 0) {
-                    return true;
-                } else {
-                    return false;
-                }
-            }
+		while (numRows < maxRows) {
+			if (resStream == null) {
+				if (numRows > 0) {
+					return true;
+				} else {
+					return false;
+				}
+			}
 
-            bos.reset();
-            Utilities.StreamStatus ss;
-            try {
-                ss = Utilities.readColumn(resStream, bos);
-                if (bos.getCount() > 0) {
-                    row = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
-                } else if (ss == Utilities.StreamStatus.TERMINATED) {
-                    row = new String();
-                }
+			bos.reset();
+			Utilities.StreamStatus ss;
+			try {
+				ss = Utilities.readColumn(resStream, bos);
+				if (bos.getCount() > 0) {
+					row = new String(bos.getData(), 0, bos.getCount(), "UTF-8");
+				} else if (ss == Utilities.StreamStatus.TERMINATED) {
+					row = new String();
+				}
 
-                if (row != null) {
-                    numRows++;
-                    res.add(row);
-                }
-            } catch (IOException e) {
-                console.printError("FAILED: Unexpected IO exception : " + e.getMessage());
-                res = null;
-                return false;
-            }
+				if (row != null) {
+					numRows++;
+					res.add(row);
+				}
+			} catch (IOException e) {
+				console.printError("FAILED: Unexpected IO exception : "
+						+ e.getMessage());
+				res = null;
+				return false;
+			}
 
-            if (ss == Utilities.StreamStatus.EOF) {
-                resStream = ctx.getStream();
-            }
-        }
-        return true;
-    }
+			if (ss == Utilities.StreamStatus.EOF) {
+				resStream = ctx.getStream();
+			}
+		}
+		return true;
+	}
 
-    public int close() {
-        try {
-            if (plan != null) {
-                FetchTask fetchTask = plan.getFetchTask();
-                if (null != fetchTask) {
-                    try {
-                        fetchTask.clearFetch();
-                    } catch (Exception e) {
-                        LOG.debug(" Exception while clearing the Fetch task ", e);
-                    }
-                }
-            }
-            if (ctx != null) {
-                ctx.clear();
-            }
-            if (null != resStream) {
-                try {
-                    ((FSDataInputStream) resStream).close();
-                } catch (Exception e) {
-                    LOG.debug(" Exception while closing the resStream ", e);
-                }
-            }
-        } catch (Exception e) {
-            console.printError("FAILED: Hive Internal Error: " + Utilities.getNameMessage(e) + "\n"
-                    + org.apache.hadoop.util.StringUtils.stringifyException(e));
-            return 13;
-        }
+	public int close() {
+		try {
+			if (plan != null) {
+				FetchTask fetchTask = plan.getFetchTask();
+				if (null != fetchTask) {
+					try {
+						fetchTask.clearFetch();
+					} catch (Exception e) {
+						LOG.debug(" Exception while clearing the Fetch task ",
+								e);
+					}
+				}
+			}
+			if (ctx != null) {
+				ctx.clear();
+			}
+			if (null != resStream) {
+				try {
+					((FSDataInputStream) resStream).close();
+				} catch (Exception e) {
+					LOG.debug(" Exception while closing the resStream ", e);
+				}
+			}
+		} catch (Exception e) {
+			console.printError("FAILED: Hive Internal Error: "
+					+ Utilities.getNameMessage(e) + "\n"
+					+ org.apache.hadoop.util.StringUtils.stringifyException(e));
+			return 13;
+		}
 
-        return 0;
-    }
+		return 0;
+	}
 
-    public void destroy() {
-        releaseLocks();
-    }
+	public void destroy() {
+		releaseLocks();
+	}
 
-    public org.apache.hadoop.hive.ql.plan.api.Query getQueryPlan() throws IOException {
-        return plan.getQueryPlan();
-    }
+	public org.apache.hadoop.hive.ql.plan.api.Query getQueryPlan()
+			throws IOException {
+		return plan.getQueryPlan();
+	}
 
-    public int getTryCount() {
-        return tryCount;
-    }
+	public int getTryCount() {
+		return tryCount;
+	}
 
-    public void setTryCount(int tryCount) {
-        this.tryCount = tryCount;
-    }
+	public void setTryCount(int tryCount) {
+		this.tryCount = tryCount;
+	}
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
index 4a05927..b174432 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
@@ -52,184 +52,202 @@
 @Description(name = "avg", value = "_FUNC_(x) - Returns the mean of a set of numbers")
 public class GenericUDAFAverage extends AbstractGenericUDAFResolver {
 
-    static final Log LOG = LogFactory.getLog(GenericUDAFAverage.class.getName());
+	static final Log LOG = LogFactory
+			.getLog(GenericUDAFAverage.class.getName());
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        if (parameters.length != 1) {
-            throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
-        }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		if (parameters.length != 1) {
+			throw new UDFArgumentTypeException(parameters.length - 1,
+					"Exactly one argument is expected.");
+		}
 
-        if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
-                    + parameters[0].getTypeName() + " is passed.");
-        }
-        switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
-            case BYTE:
-            case SHORT:
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-            case STRING:
-                return new GenericUDAFAverageEvaluator();
-            case BOOLEAN:
-            default:
-                throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
-                        + parameters[0].getTypeName() + " is passed.");
-        }
-    }
+		if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(0,
+					"Only primitive type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+		switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+		case BYTE:
+		case SHORT:
+		case INT:
+		case LONG:
+		case FLOAT:
+		case DOUBLE:
+		case STRING:
+			return new GenericUDAFAverageEvaluator();
+		case BOOLEAN:
+		default:
+			throw new UDFArgumentTypeException(0,
+					"Only numeric or string type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+	}
 
-    /**
-     * GenericUDAFAverageEvaluator.
-     * 
-     */
-    public static class GenericUDAFAverageEvaluator extends GenericUDAFEvaluator {
+	/**
+	 * GenericUDAFAverageEvaluator.
+	 * 
+	 */
+	public static class GenericUDAFAverageEvaluator extends
+			GenericUDAFEvaluator {
 
-        // For PARTIAL1 and COMPLETE
-        PrimitiveObjectInspector inputOI;
+		// For PARTIAL1 and COMPLETE
+		PrimitiveObjectInspector inputOI;
 
-        // For PARTIAL2 and FINAL
-        StructObjectInspector soi;
-        StructField countField;
-        StructField sumField;
-        LongObjectInspector countFieldOI;
-        DoubleObjectInspector sumFieldOI;
+		// For PARTIAL2 and FINAL
+		StructObjectInspector soi;
+		StructField countField;
+		StructField sumField;
+		LongObjectInspector countFieldOI;
+		DoubleObjectInspector sumFieldOI;
 
-        // For PARTIAL1 and PARTIAL2
-        Object[] partialResult;
+		// For PARTIAL1 and PARTIAL2
+		Object[] partialResult;
 
-        // For FINAL and COMPLETE
-        DoubleWritable result;
+		// For FINAL and COMPLETE
+		DoubleWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            super.init(m, parameters);
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			super.init(m, parameters);
 
-            // init input
-            if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-                inputOI = (PrimitiveObjectInspector) parameters[0];
-            } else {
-                soi = (StructObjectInspector) parameters[0];
-                countField = soi.getStructFieldRef("count");
-                sumField = soi.getStructFieldRef("sum");
-                countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
-                sumFieldOI = (DoubleObjectInspector) sumField.getFieldObjectInspector();
-            }
+			// init input
+			if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+				inputOI = (PrimitiveObjectInspector) parameters[0];
+			} else {
+				soi = (StructObjectInspector) parameters[0];
+				countField = soi.getStructFieldRef("count");
+				sumField = soi.getStructFieldRef("sum");
+				countFieldOI = (LongObjectInspector) countField
+						.getFieldObjectInspector();
+				sumFieldOI = (DoubleObjectInspector) sumField
+						.getFieldObjectInspector();
+			}
 
-            // init output
-            if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-                // The output of a partial aggregation is a struct containing
-                // a "long" count and a "double" sum.
+			// init output
+			if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+				// The output of a partial aggregation is a struct containing
+				// a "long" count and a "double" sum.
 
-                ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
-                foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                ArrayList<String> fname = new ArrayList<String>();
-                fname.add("count");
-                fname.add("sum");
-                partialResult = new Object[2];
-                partialResult[0] = new LongWritable(0);
-                partialResult[1] = new DoubleWritable(0);
-                return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+				ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+				foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				ArrayList<String> fname = new ArrayList<String>();
+				fname.add("count");
+				fname.add("sum");
+				partialResult = new Object[2];
+				partialResult[0] = new LongWritable(0);
+				partialResult[1] = new DoubleWritable(0);
+				return ObjectInspectorFactory.getStandardStructObjectInspector(
+						fname, foi);
 
-            } else {
-                result = new DoubleWritable(0);
-                return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
-            }
-        }
+			} else {
+				result = new DoubleWritable(0);
+				return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+			}
+		}
 
-        static class AverageAgg implements SerializableBuffer {
-            long count;
-            double sum;
+		static class AverageAgg implements SerializableBuffer {
+			long count;
+			double sum;
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                count = BufferSerDeUtil.getLong(data, start);
-                start += 8;
-                sum = BufferSerDeUtil.getDouble(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				count = BufferSerDeUtil.getLong(data, start);
+				start += 8;
+				sum = BufferSerDeUtil.getDouble(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeLong(count, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(sum, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeLong(count, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(sum, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeLong(count);
-                output.writeDouble(sum);
-            }
-        };
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeLong(count);
+				output.writeDouble(sum);
+			}
+		};
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            AverageAgg result = new AverageAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			AverageAgg result = new AverageAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            AverageAgg myagg = (AverageAgg) agg;
-            myagg.count = 0;
-            myagg.sum = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			AverageAgg myagg = (AverageAgg) agg;
+			myagg.count = 0;
+			myagg.sum = 0;
+		}
 
-        boolean warned = false;
+		boolean warned = false;
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            Object p = parameters[0];
-            if (p != null) {
-                AverageAgg myagg = (AverageAgg) agg;
-                try {
-                    double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
-                    myagg.count++;
-                    myagg.sum += v;
-                } catch (NumberFormatException e) {
-                    if (!warned) {
-                        warned = true;
-                        LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
-                        LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
-                    }
-                }
-            }
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			Object p = parameters[0];
+			if (p != null) {
+				AverageAgg myagg = (AverageAgg) agg;
+				try {
+					double v = PrimitiveObjectInspectorUtils.getDouble(p,
+							inputOI);
+					myagg.count++;
+					myagg.sum += v;
+				} catch (NumberFormatException e) {
+					if (!warned) {
+						warned = true;
+						LOG.warn(getClass().getSimpleName() + " "
+								+ StringUtils.stringifyException(e));
+						LOG.warn(getClass().getSimpleName()
+								+ " ignoring similar exceptions.");
+					}
+				}
+			}
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            AverageAgg myagg = (AverageAgg) agg;
-            ((LongWritable) partialResult[0]).set(myagg.count);
-            ((DoubleWritable) partialResult[1]).set(myagg.sum);
-            return partialResult;
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			AverageAgg myagg = (AverageAgg) agg;
+			((LongWritable) partialResult[0]).set(myagg.count);
+			((DoubleWritable) partialResult[1]).set(myagg.sum);
+			return partialResult;
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                AverageAgg myagg = (AverageAgg) agg;
-                Object partialCount = soi.getStructFieldData(partial, countField);
-                Object partialSum = soi.getStructFieldData(partial, sumField);
-                myagg.count += countFieldOI.get(partialCount);
-                myagg.sum += sumFieldOI.get(partialSum);
-            }
-        }
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				AverageAgg myagg = (AverageAgg) agg;
+				Object partialCount = soi.getStructFieldData(partial,
+						countField);
+				Object partialSum = soi.getStructFieldData(partial, sumField);
+				myagg.count += countFieldOI.get(partialCount);
+				myagg.sum += sumFieldOI.get(partialSum);
+			}
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            AverageAgg myagg = (AverageAgg) agg;
-            if (myagg.count == 0) {
-                return null;
-            } else {
-                result.set(myagg.sum / myagg.count);
-                return result;
-            }
-        }
-    }
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			AverageAgg myagg = (AverageAgg) agg;
+			if (myagg.count == 0) {
+				return null;
+			} else {
+				result.set(myagg.sum / myagg.count);
+				return result;
+			}
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
index 6652afc..716faac 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCorrelation.java
@@ -63,336 +63,366 @@
  * 
  */
 @Description(name = "corr", value = "_FUNC_(x,y) - Returns the Pearson coefficient of correlation\n"
-        + "between a set of number pairs", extended = "The function takes as arguments any pair of numeric types and returns a double.\n"
-        + "Any pair with a NULL is ignored. If the function is applied to an empty set or\n"
-        + "a singleton set, NULL will be returned. Otherwise, it computes the following:\n"
-        + "   COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y))\n"
-        + "where neither x nor y is null,\n"
-        + "COVAR_POP is the population covariance,\n" + "and STDDEV_POP is the population standard deviation.")
+		+ "between a set of number pairs", extended = "The function takes as arguments any pair of numeric types and returns a double.\n"
+		+ "Any pair with a NULL is ignored. If the function is applied to an empty set or\n"
+		+ "a singleton set, NULL will be returned. Otherwise, it computes the following:\n"
+		+ "   COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y))\n"
+		+ "where neither x nor y is null,\n"
+		+ "COVAR_POP is the population covariance,\n"
+		+ "and STDDEV_POP is the population standard deviation.")
 public class GenericUDAFCorrelation extends AbstractGenericUDAFResolver {
 
-    static final Log LOG = LogFactory.getLog(GenericUDAFCorrelation.class.getName());
+	static final Log LOG = LogFactory.getLog(GenericUDAFCorrelation.class
+			.getName());
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        if (parameters.length != 2) {
-            throw new UDFArgumentTypeException(parameters.length - 1, "Exactly two arguments are expected.");
-        }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		if (parameters.length != 2) {
+			throw new UDFArgumentTypeException(parameters.length - 1,
+					"Exactly two arguments are expected.");
+		}
 
-        if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
-                    + parameters[0].getTypeName() + " is passed.");
-        }
+		if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(0,
+					"Only primitive type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
 
-        if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(1, "Only primitive type arguments are accepted but "
-                    + parameters[1].getTypeName() + " is passed.");
-        }
+		if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(1,
+					"Only primitive type arguments are accepted but "
+							+ parameters[1].getTypeName() + " is passed.");
+		}
 
-        switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
-            case BYTE:
-            case SHORT:
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-                switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
-                    case BYTE:
-                    case SHORT:
-                    case INT:
-                    case LONG:
-                    case FLOAT:
-                    case DOUBLE:
-                        return new GenericUDAFCorrelationEvaluator();
-                    case STRING:
-                    case BOOLEAN:
-                    default:
-                        throw new UDFArgumentTypeException(1, "Only numeric type arguments are accepted but "
-                                + parameters[1].getTypeName() + " is passed.");
-                }
-            case STRING:
-            case BOOLEAN:
-            default:
-                throw new UDFArgumentTypeException(0, "Only numeric type arguments are accepted but "
-                        + parameters[0].getTypeName() + " is passed.");
-        }
-    }
+		switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+		case BYTE:
+		case SHORT:
+		case INT:
+		case LONG:
+		case FLOAT:
+		case DOUBLE:
+			switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
+			case BYTE:
+			case SHORT:
+			case INT:
+			case LONG:
+			case FLOAT:
+			case DOUBLE:
+				return new GenericUDAFCorrelationEvaluator();
+			case STRING:
+			case BOOLEAN:
+			default:
+				throw new UDFArgumentTypeException(1,
+						"Only numeric type arguments are accepted but "
+								+ parameters[1].getTypeName() + " is passed.");
+			}
+		case STRING:
+		case BOOLEAN:
+		default:
+			throw new UDFArgumentTypeException(0,
+					"Only numeric type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+	}
 
-    /**
-     * Evaluate the Pearson correlation coefficient using a stable one-pass
-     * algorithm, based on work by Philippe Pébay and Donald Knuth.
-     * 
-     * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
-     * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
-     * mx_(n-1))*(y_n - my_n) : <covariance * n> vx_n = vx_(n-1) + (x_n -
-     * mx_n)(x_n - mx_(n-1)): <variance * n> vy_n = vy_(n-1) + (y_n - my_n)(y_n
-     * - my_(n-1)): <variance * n>
-     * 
-     * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X vx_(A,B)
-     * = vx_A + vx_B + (mx_A - mx_B)*(mx_A - mx_B)*n_A*n_B/(n_A+n_B) vy_(A,B) =
-     * vy_A + vy_B + (my_A - my_B)*(my_A - my_B)*n_A*n_B/(n_A+n_B)
-     * 
-     */
-    public static class GenericUDAFCorrelationEvaluator extends GenericUDAFEvaluator {
+	/**
+	 * Evaluate the Pearson correlation coefficient using a stable one-pass
+	 * algorithm, based on work by Philippe Pébay and Donald Knuth.
+	 * 
+	 * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
+	 * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
+	 * mx_(n-1))*(y_n - my_n) : <covariance * n> vx_n = vx_(n-1) + (x_n -
+	 * mx_n)(x_n - mx_(n-1)): <variance * n> vy_n = vy_(n-1) + (y_n - my_n)(y_n
+	 * - my_(n-1)): <variance * n>
+	 * 
+	 * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X vx_(A,B)
+	 * = vx_A + vx_B + (mx_A - mx_B)*(mx_A - mx_B)*n_A*n_B/(n_A+n_B) vy_(A,B) =
+	 * vy_A + vy_B + (my_A - my_B)*(my_A - my_B)*n_A*n_B/(n_A+n_B)
+	 * 
+	 */
+	public static class GenericUDAFCorrelationEvaluator extends
+			GenericUDAFEvaluator {
 
-        // For PARTIAL1 and COMPLETE
-        private PrimitiveObjectInspector xInputOI;
-        private PrimitiveObjectInspector yInputOI;
+		// For PARTIAL1 and COMPLETE
+		private PrimitiveObjectInspector xInputOI;
+		private PrimitiveObjectInspector yInputOI;
 
-        // For PARTIAL2 and FINAL
-        private StructObjectInspector soi;
-        private StructField countField;
-        private StructField xavgField;
-        private StructField yavgField;
-        private StructField xvarField;
-        private StructField yvarField;
-        private StructField covarField;
-        private LongObjectInspector countFieldOI;
-        private DoubleObjectInspector xavgFieldOI;
-        private DoubleObjectInspector yavgFieldOI;
-        private DoubleObjectInspector xvarFieldOI;
-        private DoubleObjectInspector yvarFieldOI;
-        private DoubleObjectInspector covarFieldOI;
+		// For PARTIAL2 and FINAL
+		private StructObjectInspector soi;
+		private StructField countField;
+		private StructField xavgField;
+		private StructField yavgField;
+		private StructField xvarField;
+		private StructField yvarField;
+		private StructField covarField;
+		private LongObjectInspector countFieldOI;
+		private DoubleObjectInspector xavgFieldOI;
+		private DoubleObjectInspector yavgFieldOI;
+		private DoubleObjectInspector xvarFieldOI;
+		private DoubleObjectInspector yvarFieldOI;
+		private DoubleObjectInspector covarFieldOI;
 
-        // For PARTIAL1 and PARTIAL2
-        private Object[] partialResult;
+		// For PARTIAL1 and PARTIAL2
+		private Object[] partialResult;
 
-        // For FINAL and COMPLETE
-        private DoubleWritable result;
+		// For FINAL and COMPLETE
+		private DoubleWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            super.init(m, parameters);
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			super.init(m, parameters);
 
-            // init input
-            if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-                assert (parameters.length == 2);
-                xInputOI = (PrimitiveObjectInspector) parameters[0];
-                yInputOI = (PrimitiveObjectInspector) parameters[1];
-            } else {
-                assert (parameters.length == 1);
-                soi = (StructObjectInspector) parameters[0];
+			// init input
+			if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+				assert (parameters.length == 2);
+				xInputOI = (PrimitiveObjectInspector) parameters[0];
+				yInputOI = (PrimitiveObjectInspector) parameters[1];
+			} else {
+				assert (parameters.length == 1);
+				soi = (StructObjectInspector) parameters[0];
 
-                countField = soi.getStructFieldRef("count");
-                xavgField = soi.getStructFieldRef("xavg");
-                yavgField = soi.getStructFieldRef("yavg");
-                xvarField = soi.getStructFieldRef("xvar");
-                yvarField = soi.getStructFieldRef("yvar");
-                covarField = soi.getStructFieldRef("covar");
+				countField = soi.getStructFieldRef("count");
+				xavgField = soi.getStructFieldRef("xavg");
+				yavgField = soi.getStructFieldRef("yavg");
+				xvarField = soi.getStructFieldRef("xvar");
+				yvarField = soi.getStructFieldRef("yvar");
+				covarField = soi.getStructFieldRef("covar");
 
-                countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
-                xavgFieldOI = (DoubleObjectInspector) xavgField.getFieldObjectInspector();
-                yavgFieldOI = (DoubleObjectInspector) yavgField.getFieldObjectInspector();
-                xvarFieldOI = (DoubleObjectInspector) xvarField.getFieldObjectInspector();
-                yvarFieldOI = (DoubleObjectInspector) yvarField.getFieldObjectInspector();
-                covarFieldOI = (DoubleObjectInspector) covarField.getFieldObjectInspector();
-            }
+				countFieldOI = (LongObjectInspector) countField
+						.getFieldObjectInspector();
+				xavgFieldOI = (DoubleObjectInspector) xavgField
+						.getFieldObjectInspector();
+				yavgFieldOI = (DoubleObjectInspector) yavgField
+						.getFieldObjectInspector();
+				xvarFieldOI = (DoubleObjectInspector) xvarField
+						.getFieldObjectInspector();
+				yvarFieldOI = (DoubleObjectInspector) yvarField
+						.getFieldObjectInspector();
+				covarFieldOI = (DoubleObjectInspector) covarField
+						.getFieldObjectInspector();
+			}
 
-            // init output
-            if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-                // The output of a partial aggregation is a struct containing
-                // a long count, two double averages, two double variances,
-                // and a double covariance.
+			// init output
+			if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+				// The output of a partial aggregation is a struct containing
+				// a long count, two double averages, two double variances,
+				// and a double covariance.
 
-                ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+				ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
 
-                foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
 
-                ArrayList<String> fname = new ArrayList<String>();
-                fname.add("count");
-                fname.add("xavg");
-                fname.add("yavg");
-                fname.add("xvar");
-                fname.add("yvar");
-                fname.add("covar");
+				ArrayList<String> fname = new ArrayList<String>();
+				fname.add("count");
+				fname.add("xavg");
+				fname.add("yavg");
+				fname.add("xvar");
+				fname.add("yvar");
+				fname.add("covar");
 
-                partialResult = new Object[6];
-                partialResult[0] = new LongWritable(0);
-                partialResult[1] = new DoubleWritable(0);
-                partialResult[2] = new DoubleWritable(0);
-                partialResult[3] = new DoubleWritable(0);
-                partialResult[4] = new DoubleWritable(0);
-                partialResult[5] = new DoubleWritable(0);
+				partialResult = new Object[6];
+				partialResult[0] = new LongWritable(0);
+				partialResult[1] = new DoubleWritable(0);
+				partialResult[2] = new DoubleWritable(0);
+				partialResult[3] = new DoubleWritable(0);
+				partialResult[4] = new DoubleWritable(0);
+				partialResult[5] = new DoubleWritable(0);
 
-                return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+				return ObjectInspectorFactory.getStandardStructObjectInspector(
+						fname, foi);
 
-            } else {
-                setResult(new DoubleWritable(0));
-                return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
-            }
-        }
+			} else {
+				setResult(new DoubleWritable(0));
+				return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+			}
+		}
 
-        static class StdAgg implements SerializableBuffer {
-            long count; // number n of elements
-            double xavg; // average of x elements
-            double yavg; // average of y elements
-            double xvar; // n times the variance of x elements
-            double yvar; // n times the variance of y elements
-            double covar; // n times the covariance
+		static class StdAgg implements SerializableBuffer {
+			long count; // number n of elements
+			double xavg; // average of x elements
+			double yavg; // average of y elements
+			double xvar; // n times the variance of x elements
+			double yvar; // n times the variance of y elements
+			double covar; // n times the covariance
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                count = BufferSerDeUtil.getLong(data, start);
-                start += 8;
-                xavg = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                yavg = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                xvar = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                yvar = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                covar = BufferSerDeUtil.getDouble(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				count = BufferSerDeUtil.getLong(data, start);
+				start += 8;
+				xavg = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				yavg = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				xvar = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				yvar = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				covar = BufferSerDeUtil.getDouble(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeLong(count, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(xavg, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(yavg, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(xvar, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(yvar, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(covar, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeLong(count, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(xavg, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(yavg, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(xvar, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(yvar, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(covar, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeLong(count);
-                output.writeDouble(xavg);
-                output.writeDouble(yavg);
-                output.writeDouble(xvar);
-                output.writeDouble(yvar);
-                output.writeDouble(covar);
-            }
-        };
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeLong(count);
+				output.writeDouble(xavg);
+				output.writeDouble(yavg);
+				output.writeDouble(xvar);
+				output.writeDouble(yvar);
+				output.writeDouble(covar);
+			}
+		};
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            StdAgg result = new StdAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			StdAgg result = new StdAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            myagg.count = 0;
-            myagg.xavg = 0;
-            myagg.yavg = 0;
-            myagg.xvar = 0;
-            myagg.yvar = 0;
-            myagg.covar = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			myagg.count = 0;
+			myagg.xavg = 0;
+			myagg.yavg = 0;
+			myagg.xvar = 0;
+			myagg.yvar = 0;
+			myagg.covar = 0;
+		}
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 2);
-            Object px = parameters[0];
-            Object py = parameters[1];
-            if (px != null && py != null) {
-                StdAgg myagg = (StdAgg) agg;
-                double vx = PrimitiveObjectInspectorUtils.getDouble(px, xInputOI);
-                double vy = PrimitiveObjectInspectorUtils.getDouble(py, yInputOI);
-                double xavgOld = myagg.xavg;
-                double yavgOld = myagg.yavg;
-                myagg.count++;
-                myagg.xavg += (vx - xavgOld) / myagg.count;
-                myagg.yavg += (vy - yavgOld) / myagg.count;
-                if (myagg.count > 1) {
-                    myagg.covar += (vx - xavgOld) * (vy - myagg.yavg);
-                    myagg.xvar += (vx - xavgOld) * (vx - myagg.xavg);
-                    myagg.yvar += (vy - yavgOld) * (vy - myagg.yavg);
-                }
-            }
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 2);
+			Object px = parameters[0];
+			Object py = parameters[1];
+			if (px != null && py != null) {
+				StdAgg myagg = (StdAgg) agg;
+				double vx = PrimitiveObjectInspectorUtils.getDouble(px,
+						xInputOI);
+				double vy = PrimitiveObjectInspectorUtils.getDouble(py,
+						yInputOI);
+				double xavgOld = myagg.xavg;
+				double yavgOld = myagg.yavg;
+				myagg.count++;
+				myagg.xavg += (vx - xavgOld) / myagg.count;
+				myagg.yavg += (vy - yavgOld) / myagg.count;
+				if (myagg.count > 1) {
+					myagg.covar += (vx - xavgOld) * (vy - myagg.yavg);
+					myagg.xvar += (vx - xavgOld) * (vx - myagg.xavg);
+					myagg.yvar += (vy - yavgOld) * (vy - myagg.yavg);
+				}
+			}
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            ((LongWritable) partialResult[0]).set(myagg.count);
-            ((DoubleWritable) partialResult[1]).set(myagg.xavg);
-            ((DoubleWritable) partialResult[2]).set(myagg.yavg);
-            ((DoubleWritable) partialResult[3]).set(myagg.xvar);
-            ((DoubleWritable) partialResult[4]).set(myagg.yvar);
-            ((DoubleWritable) partialResult[5]).set(myagg.covar);
-            return partialResult;
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			((LongWritable) partialResult[0]).set(myagg.count);
+			((DoubleWritable) partialResult[1]).set(myagg.xavg);
+			((DoubleWritable) partialResult[2]).set(myagg.yavg);
+			((DoubleWritable) partialResult[3]).set(myagg.xvar);
+			((DoubleWritable) partialResult[4]).set(myagg.yvar);
+			((DoubleWritable) partialResult[5]).set(myagg.covar);
+			return partialResult;
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                StdAgg myagg = (StdAgg) agg;
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				StdAgg myagg = (StdAgg) agg;
 
-                Object partialCount = soi.getStructFieldData(partial, countField);
-                Object partialXAvg = soi.getStructFieldData(partial, xavgField);
-                Object partialYAvg = soi.getStructFieldData(partial, yavgField);
-                Object partialXVar = soi.getStructFieldData(partial, xvarField);
-                Object partialYVar = soi.getStructFieldData(partial, yvarField);
-                Object partialCovar = soi.getStructFieldData(partial, covarField);
+				Object partialCount = soi.getStructFieldData(partial,
+						countField);
+				Object partialXAvg = soi.getStructFieldData(partial, xavgField);
+				Object partialYAvg = soi.getStructFieldData(partial, yavgField);
+				Object partialXVar = soi.getStructFieldData(partial, xvarField);
+				Object partialYVar = soi.getStructFieldData(partial, yvarField);
+				Object partialCovar = soi.getStructFieldData(partial,
+						covarField);
 
-                long nA = myagg.count;
-                long nB = countFieldOI.get(partialCount);
+				long nA = myagg.count;
+				long nB = countFieldOI.get(partialCount);
 
-                if (nA == 0) {
-                    // Just copy the information since there is nothing so far
-                    myagg.count = countFieldOI.get(partialCount);
-                    myagg.xavg = xavgFieldOI.get(partialXAvg);
-                    myagg.yavg = yavgFieldOI.get(partialYAvg);
-                    myagg.xvar = xvarFieldOI.get(partialXVar);
-                    myagg.yvar = yvarFieldOI.get(partialYVar);
-                    myagg.covar = covarFieldOI.get(partialCovar);
-                }
+				if (nA == 0) {
+					// Just copy the information since there is nothing so far
+					myagg.count = countFieldOI.get(partialCount);
+					myagg.xavg = xavgFieldOI.get(partialXAvg);
+					myagg.yavg = yavgFieldOI.get(partialYAvg);
+					myagg.xvar = xvarFieldOI.get(partialXVar);
+					myagg.yvar = yvarFieldOI.get(partialYVar);
+					myagg.covar = covarFieldOI.get(partialCovar);
+				}
 
-                if (nA != 0 && nB != 0) {
-                    // Merge the two partials
-                    double xavgA = myagg.xavg;
-                    double yavgA = myagg.yavg;
-                    double xavgB = xavgFieldOI.get(partialXAvg);
-                    double yavgB = yavgFieldOI.get(partialYAvg);
-                    double xvarB = xvarFieldOI.get(partialXVar);
-                    double yvarB = yvarFieldOI.get(partialYVar);
-                    double covarB = covarFieldOI.get(partialCovar);
+				if (nA != 0 && nB != 0) {
+					// Merge the two partials
+					double xavgA = myagg.xavg;
+					double yavgA = myagg.yavg;
+					double xavgB = xavgFieldOI.get(partialXAvg);
+					double yavgB = yavgFieldOI.get(partialYAvg);
+					double xvarB = xvarFieldOI.get(partialXVar);
+					double yvarB = yvarFieldOI.get(partialYVar);
+					double covarB = covarFieldOI.get(partialCovar);
 
-                    myagg.count += nB;
-                    myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
-                    myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
-                    myagg.xvar += xvarB + (xavgA - xavgB) * (xavgA - xavgB) * myagg.count;
-                    myagg.yvar += yvarB + (yavgA - yavgB) * (yavgA - yavgB) * myagg.count;
-                    myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB) * ((double) (nA * nB) / myagg.count);
-                }
-            }
-        }
+					myagg.count += nB;
+					myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
+					myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
+					myagg.xvar += xvarB + (xavgA - xavgB) * (xavgA - xavgB)
+							* myagg.count;
+					myagg.yvar += yvarB + (yavgA - yavgB) * (yavgA - yavgB)
+							* myagg.count;
+					myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB)
+							* ((double) (nA * nB) / myagg.count);
+				}
+			}
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
 
-            if (myagg.count < 2) { // SQL standard - return null for zero or one
-                                   // pair
-                return null;
-            } else {
-                getResult().set(myagg.covar / java.lang.Math.sqrt(myagg.xvar) / java.lang.Math.sqrt(myagg.yvar));
-                return getResult();
-            }
-        }
+			if (myagg.count < 2) { // SQL standard - return null for zero or one
+									// pair
+				return null;
+			} else {
+				getResult().set(
+						myagg.covar / java.lang.Math.sqrt(myagg.xvar)
+								/ java.lang.Math.sqrt(myagg.yvar));
+				return getResult();
+			}
+		}
 
-        public void setResult(DoubleWritable result) {
-            this.result = result;
-        }
+		public void setResult(DoubleWritable result) {
+			this.result = result;
+		}
 
-        public DoubleWritable getResult() {
-            return result;
-        }
-    }
+		public DoubleWritable getResult() {
+			return result;
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
index e4081f5..4160d5b 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
@@ -20,8 +20,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -39,137 +37,146 @@
  * This class implements the COUNT aggregation function as in SQL.
  */
 @Description(name = "count", value = "_FUNC_(*) - Returns the total number of retrieved rows, including "
-        + "rows containing NULL values.\n"
+		+ "rows containing NULL values.\n"
 
-        + "_FUNC_(expr) - Returns the number of rows for which the supplied " + "expression is non-NULL.\n"
+		+ "_FUNC_(expr) - Returns the number of rows for which the supplied "
+		+ "expression is non-NULL.\n"
 
-        + "_FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for "
-        + "which the supplied expression(s) are unique and non-NULL.")
+		+ "_FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for "
+		+ "which the supplied expression(s) are unique and non-NULL.")
 public class GenericUDAFCount implements GenericUDAFResolver2 {
 
-    private static final Log LOG = LogFactory.getLog(GenericUDAFCount.class.getName());
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		// This method implementation is preserved for backward compatibility.
+		return new GenericUDAFCountEvaluator();
+	}
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        // This method implementation is preserved for backward compatibility.
-        return new GenericUDAFCountEvaluator();
-    }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo paramInfo)
+			throws SemanticException {
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo paramInfo) throws SemanticException {
+		TypeInfo[] parameters = paramInfo.getParameters();
 
-        TypeInfo[] parameters = paramInfo.getParameters();
+		if (parameters.length == 0) {
+			if (!paramInfo.isAllColumns()) {
+				throw new UDFArgumentException("Argument expected");
+			}
+			assert !paramInfo.isDistinct() : "DISTINCT not supported with *";
+		} else {
+			if (parameters.length > 1 && !paramInfo.isDistinct()) {
+				throw new UDFArgumentException(
+						"DISTINCT keyword must be specified");
+			}
+			assert !paramInfo.isAllColumns() : "* not supported in expression list";
+		}
 
-        if (parameters.length == 0) {
-            if (!paramInfo.isAllColumns()) {
-                throw new UDFArgumentException("Argument expected");
-            }
-            assert !paramInfo.isDistinct() : "DISTINCT not supported with *";
-        } else {
-            if (parameters.length > 1 && !paramInfo.isDistinct()) {
-                throw new UDFArgumentException("DISTINCT keyword must be specified");
-            }
-            assert !paramInfo.isAllColumns() : "* not supported in expression list";
-        }
+		return new GenericUDAFCountEvaluator().setCountAllColumns(paramInfo
+				.isAllColumns());
+	}
 
-        return new GenericUDAFCountEvaluator().setCountAllColumns(paramInfo.isAllColumns());
-    }
+	/**
+	 * GenericUDAFCountEvaluator.
+	 * 
+	 */
+	public static class GenericUDAFCountEvaluator extends GenericUDAFEvaluator {
+		private boolean countAllColumns = false;
+		private LongObjectInspector partialCountAggOI;
+		private LongWritable result;
 
-    /**
-     * GenericUDAFCountEvaluator.
-     * 
-     */
-    public static class GenericUDAFCountEvaluator extends GenericUDAFEvaluator {
-        private boolean countAllColumns = false;
-        private LongObjectInspector partialCountAggOI;
-        private LongWritable result;
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			super.init(m, parameters);
+			partialCountAggOI = PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+			result = new LongWritable(0);
+			return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+		}
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            super.init(m, parameters);
-            partialCountAggOI = PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-            result = new LongWritable(0);
-            return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-        }
+		private GenericUDAFCountEvaluator setCountAllColumns(
+				boolean countAllCols) {
+			countAllColumns = countAllCols;
+			return this;
+		}
 
-        private GenericUDAFCountEvaluator setCountAllColumns(boolean countAllCols) {
-            countAllColumns = countAllCols;
-            return this;
-        }
+		/** class for storing count value. */
+		static class CountAgg implements SerializableBuffer {
+			long value;
 
-        /** class for storing count value. */
-        static class CountAgg implements SerializableBuffer {
-            long value;
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				value = BufferSerDeUtil.getLong(data, start);
+			}
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                value = BufferSerDeUtil.getLong(data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeLong(value, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeLong(value, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeLong(value);
+			}
+		}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeLong(value);
-            }
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			CountAgg buffer = new CountAgg();
+			reset(buffer);
+			return buffer;
+		}
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            CountAgg buffer = new CountAgg();
-            reset(buffer);
-            return buffer;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			((CountAgg) agg).value = 0;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            ((CountAgg) agg).value = 0;
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			// parameters == null means the input table/split is empty
+			if (parameters == null) {
+				return;
+			}
+			if (countAllColumns) {
+				assert parameters.length == 0;
+				((CountAgg) agg).value++;
+			} else {
+				assert parameters.length > 0;
+				boolean countThisRow = true;
+				for (Object nextParam : parameters) {
+					if (nextParam == null) {
+						countThisRow = false;
+						break;
+					}
+				}
+				if (countThisRow) {
+					((CountAgg) agg).value++;
+				}
+			}
+		}
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            // parameters == null means the input table/split is empty
-            if (parameters == null) {
-                return;
-            }
-            if (countAllColumns) {
-                assert parameters.length == 0;
-                ((CountAgg) agg).value++;
-            } else {
-                assert parameters.length > 0;
-                boolean countThisRow = true;
-                for (Object nextParam : parameters) {
-                    if (nextParam == null) {
-                        countThisRow = false;
-                        break;
-                    }
-                }
-                if (countThisRow) {
-                    ((CountAgg) agg).value++;
-                }
-            }
-        }
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				long p = partialCountAggOI.get(partial);
+				((CountAgg) agg).value += p;
+			}
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                long p = partialCountAggOI.get(partial);
-                ((CountAgg) agg).value += p;
-            }
-        }
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			result.set(((CountAgg) agg).value);
+			return result;
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            result.set(((CountAgg) agg).value);
-            return result;
-        }
-
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            return terminate(agg);
-        }
-    }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			return terminate(agg);
+		}
+	}
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
index 0453c3b..11d9dc3 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
@@ -57,294 +57,316 @@
  * 
  */
 @Description(name = "covariance,covar_pop", value = "_FUNC_(x,y) - Returns the population covariance of a set of number pairs", extended = "The function takes as arguments any pair of numeric types and returns a double.\n"
-        + "Any pair with a NULL is ignored. If the function is applied to an empty set, NULL\n"
-        + "will be returned. Otherwise, it computes the following:\n"
-        + "   (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y)\n" + "where neither x nor y is null.")
+		+ "Any pair with a NULL is ignored. If the function is applied to an empty set, NULL\n"
+		+ "will be returned. Otherwise, it computes the following:\n"
+		+ "   (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y)\n"
+		+ "where neither x nor y is null.")
 public class GenericUDAFCovariance extends AbstractGenericUDAFResolver {
 
-    static final Log LOG = LogFactory.getLog(GenericUDAFCovariance.class.getName());
+	static final Log LOG = LogFactory.getLog(GenericUDAFCovariance.class
+			.getName());
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        if (parameters.length != 2) {
-            throw new UDFArgumentTypeException(parameters.length - 1, "Exactly two arguments are expected.");
-        }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		if (parameters.length != 2) {
+			throw new UDFArgumentTypeException(parameters.length - 1,
+					"Exactly two arguments are expected.");
+		}
 
-        if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
-                    + parameters[0].getTypeName() + " is passed.");
-        }
+		if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(0,
+					"Only primitive type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
 
-        if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(1, "Only primitive type arguments are accepted but "
-                    + parameters[1].getTypeName() + " is passed.");
-        }
+		if (parameters[1].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(1,
+					"Only primitive type arguments are accepted but "
+							+ parameters[1].getTypeName() + " is passed.");
+		}
 
-        switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
-            case BYTE:
-            case SHORT:
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-                switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
-                    case BYTE:
-                    case SHORT:
-                    case INT:
-                    case LONG:
-                    case FLOAT:
-                    case DOUBLE:
-                        return new GenericUDAFCovarianceEvaluator();
-                    case STRING:
-                    case BOOLEAN:
-                    default:
-                        throw new UDFArgumentTypeException(1, "Only numeric or string type arguments are accepted but "
-                                + parameters[1].getTypeName() + " is passed.");
-                }
-            case STRING:
-            case BOOLEAN:
-            default:
-                throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
-                        + parameters[0].getTypeName() + " is passed.");
-        }
-    }
+		switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+		case BYTE:
+		case SHORT:
+		case INT:
+		case LONG:
+		case FLOAT:
+		case DOUBLE:
+			switch (((PrimitiveTypeInfo) parameters[1]).getPrimitiveCategory()) {
+			case BYTE:
+			case SHORT:
+			case INT:
+			case LONG:
+			case FLOAT:
+			case DOUBLE:
+				return new GenericUDAFCovarianceEvaluator();
+			case STRING:
+			case BOOLEAN:
+			default:
+				throw new UDFArgumentTypeException(1,
+						"Only numeric or string type arguments are accepted but "
+								+ parameters[1].getTypeName() + " is passed.");
+			}
+		case STRING:
+		case BOOLEAN:
+		default:
+			throw new UDFArgumentTypeException(0,
+					"Only numeric or string type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+	}
 
-    /**
-     * Evaluate the variance using the algorithm described in
-     * http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance,
-     * presumably by Pébay, Philippe (2008), in "Formulas for Robust, One-Pass
-     * Parallel Computation of Covariances and Arbitrary-Order Statistical
-     * Moments", Technical Report SAND2008-6212, Sandia National Laboratories,
-     * http://infoserve.sandia.gov/sand_doc/2008/086212.pdf
-     * 
-     * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
-     * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
-     * mx_(n-1))*(y_n - my_n) : <covariance * n>
-     * 
-     * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X
-     * 
-     * This one-pass algorithm is stable.
-     * 
-     */
-    public static class GenericUDAFCovarianceEvaluator extends GenericUDAFEvaluator {
+	/**
+	 * Evaluate the variance using the algorithm described in
+	 * http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance,
+	 * presumably by Pébay, Philippe (2008), in "Formulas for Robust, One-Pass
+	 * Parallel Computation of Covariances and Arbitrary-Order Statistical
+	 * Moments", Technical Report SAND2008-6212, Sandia National Laboratories,
+	 * http://infoserve.sandia.gov/sand_doc/2008/086212.pdf
+	 * 
+	 * Incremental: n : <count> mx_n = mx_(n-1) + [x_n - mx_(n-1)]/n : <xavg>
+	 * my_n = my_(n-1) + [y_n - my_(n-1)]/n : <yavg> c_n = c_(n-1) + (x_n -
+	 * mx_(n-1))*(y_n - my_n) : <covariance * n>
+	 * 
+	 * Merge: c_X = c_A + c_B + (mx_A - mx_B)*(my_A - my_B)*n_A*n_B/n_X
+	 * 
+	 * This one-pass algorithm is stable.
+	 * 
+	 */
+	public static class GenericUDAFCovarianceEvaluator extends
+			GenericUDAFEvaluator {
 
-        // For PARTIAL1 and COMPLETE
-        private PrimitiveObjectInspector xInputOI;
-        private PrimitiveObjectInspector yInputOI;
+		// For PARTIAL1 and COMPLETE
+		private PrimitiveObjectInspector xInputOI;
+		private PrimitiveObjectInspector yInputOI;
 
-        // For PARTIAL2 and FINAL
-        private StructObjectInspector soi;
-        private StructField countField;
-        private StructField xavgField;
-        private StructField yavgField;
-        private StructField covarField;
-        private LongObjectInspector countFieldOI;
-        private DoubleObjectInspector xavgFieldOI;
-        private DoubleObjectInspector yavgFieldOI;
-        private DoubleObjectInspector covarFieldOI;
+		// For PARTIAL2 and FINAL
+		private StructObjectInspector soi;
+		private StructField countField;
+		private StructField xavgField;
+		private StructField yavgField;
+		private StructField covarField;
+		private LongObjectInspector countFieldOI;
+		private DoubleObjectInspector xavgFieldOI;
+		private DoubleObjectInspector yavgFieldOI;
+		private DoubleObjectInspector covarFieldOI;
 
-        // For PARTIAL1 and PARTIAL2
-        private Object[] partialResult;
+		// For PARTIAL1 and PARTIAL2
+		private Object[] partialResult;
 
-        // For FINAL and COMPLETE
-        private DoubleWritable result;
+		// For FINAL and COMPLETE
+		private DoubleWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            super.init(m, parameters);
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			super.init(m, parameters);
 
-            // init input
-            if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-                assert (parameters.length == 2);
-                xInputOI = (PrimitiveObjectInspector) parameters[0];
-                yInputOI = (PrimitiveObjectInspector) parameters[1];
-            } else {
-                assert (parameters.length == 1);
-                soi = (StructObjectInspector) parameters[0];
+			// init input
+			if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+				assert (parameters.length == 2);
+				xInputOI = (PrimitiveObjectInspector) parameters[0];
+				yInputOI = (PrimitiveObjectInspector) parameters[1];
+			} else {
+				assert (parameters.length == 1);
+				soi = (StructObjectInspector) parameters[0];
 
-                countField = soi.getStructFieldRef("count");
-                xavgField = soi.getStructFieldRef("xavg");
-                yavgField = soi.getStructFieldRef("yavg");
-                covarField = soi.getStructFieldRef("covar");
+				countField = soi.getStructFieldRef("count");
+				xavgField = soi.getStructFieldRef("xavg");
+				yavgField = soi.getStructFieldRef("yavg");
+				covarField = soi.getStructFieldRef("covar");
 
-                countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
-                xavgFieldOI = (DoubleObjectInspector) xavgField.getFieldObjectInspector();
-                yavgFieldOI = (DoubleObjectInspector) yavgField.getFieldObjectInspector();
-                covarFieldOI = (DoubleObjectInspector) covarField.getFieldObjectInspector();
-            }
+				countFieldOI = (LongObjectInspector) countField
+						.getFieldObjectInspector();
+				xavgFieldOI = (DoubleObjectInspector) xavgField
+						.getFieldObjectInspector();
+				yavgFieldOI = (DoubleObjectInspector) yavgField
+						.getFieldObjectInspector();
+				covarFieldOI = (DoubleObjectInspector) covarField
+						.getFieldObjectInspector();
+			}
 
-            // init output
-            if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-                // The output of a partial aggregation is a struct containing
-                // a long count, two double averages, and a double covariance.
+			// init output
+			if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+				// The output of a partial aggregation is a struct containing
+				// a long count, two double averages, and a double covariance.
 
-                ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+				ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
 
-                foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
 
-                ArrayList<String> fname = new ArrayList<String>();
-                fname.add("count");
-                fname.add("xavg");
-                fname.add("yavg");
-                fname.add("covar");
+				ArrayList<String> fname = new ArrayList<String>();
+				fname.add("count");
+				fname.add("xavg");
+				fname.add("yavg");
+				fname.add("covar");
 
-                partialResult = new Object[4];
-                partialResult[0] = new LongWritable(0);
-                partialResult[1] = new DoubleWritable(0);
-                partialResult[2] = new DoubleWritable(0);
-                partialResult[3] = new DoubleWritable(0);
+				partialResult = new Object[4];
+				partialResult[0] = new LongWritable(0);
+				partialResult[1] = new DoubleWritable(0);
+				partialResult[2] = new DoubleWritable(0);
+				partialResult[3] = new DoubleWritable(0);
 
-                return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+				return ObjectInspectorFactory.getStandardStructObjectInspector(
+						fname, foi);
 
-            } else {
-                setResult(new DoubleWritable(0));
-                return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
-            }
-        }
+			} else {
+				setResult(new DoubleWritable(0));
+				return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+			}
+		}
 
-        static class StdAgg implements SerializableBuffer {
-            long count; // number n of elements
-            double xavg; // average of x elements
-            double yavg; // average of y elements
-            double covar; // n times the covariance
+		static class StdAgg implements SerializableBuffer {
+			long count; // number n of elements
+			double xavg; // average of x elements
+			double yavg; // average of y elements
+			double covar; // n times the covariance
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                count = BufferSerDeUtil.getLong(data, start);
-                start += 8;
-                xavg = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                yavg = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                covar = BufferSerDeUtil.getDouble(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				count = BufferSerDeUtil.getLong(data, start);
+				start += 8;
+				xavg = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				yavg = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				covar = BufferSerDeUtil.getDouble(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeLong(count, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(xavg, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(yavg, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(covar, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeLong(count, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(xavg, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(yavg, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(covar, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeLong(count);
-                output.writeDouble(xavg);
-                output.writeDouble(yavg);
-                output.writeDouble(covar);
-            }
-        };
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeLong(count);
+				output.writeDouble(xavg);
+				output.writeDouble(yavg);
+				output.writeDouble(covar);
+			}
+		};
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            StdAgg result = new StdAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			StdAgg result = new StdAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            myagg.count = 0;
-            myagg.xavg = 0;
-            myagg.yavg = 0;
-            myagg.covar = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			myagg.count = 0;
+			myagg.xavg = 0;
+			myagg.yavg = 0;
+			myagg.covar = 0;
+		}
 
-        private boolean warned = false;
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 2);
+			Object px = parameters[0];
+			Object py = parameters[1];
+			if (px != null && py != null) {
+				StdAgg myagg = (StdAgg) agg;
+				double vx = PrimitiveObjectInspectorUtils.getDouble(px,
+						xInputOI);
+				double vy = PrimitiveObjectInspectorUtils.getDouble(py,
+						yInputOI);
+				myagg.count++;
+				myagg.yavg = myagg.yavg + (vy - myagg.yavg) / myagg.count;
+				if (myagg.count > 1) {
+					myagg.covar += (vx - myagg.xavg) * (vy - myagg.yavg);
+				}
+				myagg.xavg = myagg.xavg + (vx - myagg.xavg) / myagg.count;
+			}
+		}
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 2);
-            Object px = parameters[0];
-            Object py = parameters[1];
-            if (px != null && py != null) {
-                StdAgg myagg = (StdAgg) agg;
-                double vx = PrimitiveObjectInspectorUtils.getDouble(px, xInputOI);
-                double vy = PrimitiveObjectInspectorUtils.getDouble(py, yInputOI);
-                myagg.count++;
-                myagg.yavg = myagg.yavg + (vy - myagg.yavg) / myagg.count;
-                if (myagg.count > 1) {
-                    myagg.covar += (vx - myagg.xavg) * (vy - myagg.yavg);
-                }
-                myagg.xavg = myagg.xavg + (vx - myagg.xavg) / myagg.count;
-            }
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			((LongWritable) partialResult[0]).set(myagg.count);
+			((DoubleWritable) partialResult[1]).set(myagg.xavg);
+			((DoubleWritable) partialResult[2]).set(myagg.yavg);
+			((DoubleWritable) partialResult[3]).set(myagg.covar);
+			return partialResult;
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            ((LongWritable) partialResult[0]).set(myagg.count);
-            ((DoubleWritable) partialResult[1]).set(myagg.xavg);
-            ((DoubleWritable) partialResult[2]).set(myagg.yavg);
-            ((DoubleWritable) partialResult[3]).set(myagg.covar);
-            return partialResult;
-        }
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				StdAgg myagg = (StdAgg) agg;
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                StdAgg myagg = (StdAgg) agg;
+				Object partialCount = soi.getStructFieldData(partial,
+						countField);
+				Object partialXAvg = soi.getStructFieldData(partial, xavgField);
+				Object partialYAvg = soi.getStructFieldData(partial, yavgField);
+				Object partialCovar = soi.getStructFieldData(partial,
+						covarField);
 
-                Object partialCount = soi.getStructFieldData(partial, countField);
-                Object partialXAvg = soi.getStructFieldData(partial, xavgField);
-                Object partialYAvg = soi.getStructFieldData(partial, yavgField);
-                Object partialCovar = soi.getStructFieldData(partial, covarField);
+				long nA = myagg.count;
+				long nB = countFieldOI.get(partialCount);
 
-                long nA = myagg.count;
-                long nB = countFieldOI.get(partialCount);
+				if (nA == 0) {
+					// Just copy the information since there is nothing so far
+					myagg.count = countFieldOI.get(partialCount);
+					myagg.xavg = xavgFieldOI.get(partialXAvg);
+					myagg.yavg = yavgFieldOI.get(partialYAvg);
+					myagg.covar = covarFieldOI.get(partialCovar);
+				}
 
-                if (nA == 0) {
-                    // Just copy the information since there is nothing so far
-                    myagg.count = countFieldOI.get(partialCount);
-                    myagg.xavg = xavgFieldOI.get(partialXAvg);
-                    myagg.yavg = yavgFieldOI.get(partialYAvg);
-                    myagg.covar = covarFieldOI.get(partialCovar);
-                }
+				if (nA != 0 && nB != 0) {
+					// Merge the two partials
+					double xavgA = myagg.xavg;
+					double yavgA = myagg.yavg;
+					double xavgB = xavgFieldOI.get(partialXAvg);
+					double yavgB = yavgFieldOI.get(partialYAvg);
+					double covarB = covarFieldOI.get(partialCovar);
 
-                if (nA != 0 && nB != 0) {
-                    // Merge the two partials
-                    double xavgA = myagg.xavg;
-                    double yavgA = myagg.yavg;
-                    double xavgB = xavgFieldOI.get(partialXAvg);
-                    double yavgB = yavgFieldOI.get(partialYAvg);
-                    double covarB = covarFieldOI.get(partialCovar);
+					myagg.count += nB;
+					myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
+					myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
+					myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB)
+							* ((double) (nA * nB) / myagg.count);
+				}
+			}
+		}
 
-                    myagg.count += nB;
-                    myagg.xavg = (xavgA * nA + xavgB * nB) / myagg.count;
-                    myagg.yavg = (yavgA * nA + yavgB * nB) / myagg.count;
-                    myagg.covar += covarB + (xavgA - xavgB) * (yavgA - yavgB) * ((double) (nA * nB) / myagg.count);
-                }
-            }
-        }
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
+			if (myagg.count == 0) { // SQL standard - return null for zero
+									// elements
+				return null;
+			} else {
+				getResult().set(myagg.covar / (myagg.count));
+				return getResult();
+			}
+		}
 
-            if (myagg.count == 0) { // SQL standard - return null for zero
-                                    // elements
-                return null;
-            } else {
-                getResult().set(myagg.covar / (myagg.count));
-                return getResult();
-            }
-        }
+		public void setResult(DoubleWritable result) {
+			this.result = result;
+		}
 
-        public void setResult(DoubleWritable result) {
-            this.result = result;
-        }
-
-        public DoubleWritable getResult() {
-            return result;
-        }
-    }
+		public DoubleWritable getResult() {
+			return result;
+		}
+	}
 
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
index 23b453f..0323531 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
@@ -46,230 +46,249 @@
 @Description(name = "sum", value = "_FUNC_(x) - Returns the sum of a set of numbers")
 public class GenericUDAFSum extends AbstractGenericUDAFResolver {
 
-    static final Log LOG = LogFactory.getLog(GenericUDAFSum.class.getName());
+	static final Log LOG = LogFactory.getLog(GenericUDAFSum.class.getName());
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        if (parameters.length != 1) {
-            throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
-        }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		if (parameters.length != 1) {
+			throw new UDFArgumentTypeException(parameters.length - 1,
+					"Exactly one argument is expected.");
+		}
 
-        if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
-                    + parameters[0].getTypeName() + " is passed.");
-        }
-        switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
-            case BYTE:
-            case SHORT:
-            case INT:
-            case LONG:
-                return new GenericUDAFSumLong();
-            case FLOAT:
-            case DOUBLE:
-            case STRING:
-                return new GenericUDAFSumDouble();
-            case BOOLEAN:
-            default:
-                throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
-                        + parameters[0].getTypeName() + " is passed.");
-        }
-    }
+		if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(0,
+					"Only primitive type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+		switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+		case BYTE:
+		case SHORT:
+		case INT:
+		case LONG:
+			return new GenericUDAFSumLong();
+		case FLOAT:
+		case DOUBLE:
+		case STRING:
+			return new GenericUDAFSumDouble();
+		case BOOLEAN:
+		default:
+			throw new UDFArgumentTypeException(0,
+					"Only numeric or string type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+	}
 
-    /**
-     * GenericUDAFSumDouble.
-     * 
-     */
-    public static class GenericUDAFSumDouble extends GenericUDAFEvaluator {
-        private PrimitiveObjectInspector inputOI;
-        private DoubleWritable result;
+	/**
+	 * GenericUDAFSumDouble.
+	 * 
+	 */
+	public static class GenericUDAFSumDouble extends GenericUDAFEvaluator {
+		private PrimitiveObjectInspector inputOI;
+		private DoubleWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            super.init(m, parameters);
-            result = new DoubleWritable(0);
-            inputOI = (PrimitiveObjectInspector) parameters[0];
-            return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
-        }
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			super.init(m, parameters);
+			result = new DoubleWritable(0);
+			inputOI = (PrimitiveObjectInspector) parameters[0];
+			return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+		}
 
-        /** class for storing double sum value. */
-        static class SumDoubleAgg implements SerializableBuffer {
-            boolean empty;
-            double sum;
+		/** class for storing double sum value. */
+		static class SumDoubleAgg implements SerializableBuffer {
+			boolean empty;
+			double sum;
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                empty = BufferSerDeUtil.getBoolean(data, start);
-                start += 1;
-                sum = BufferSerDeUtil.getDouble(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				empty = BufferSerDeUtil.getBoolean(data, start);
+				start += 1;
+				sum = BufferSerDeUtil.getDouble(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeBoolean(empty, data, start);
-                start += 1;
-                BufferSerDeUtil.writeDouble(sum, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeBoolean(empty, data, start);
+				start += 1;
+				BufferSerDeUtil.writeDouble(sum, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeBoolean(empty);
-                output.writeDouble(sum);
-            }
-        }
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeBoolean(empty);
+				output.writeDouble(sum);
+			}
+		}
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            SumDoubleAgg result = new SumDoubleAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			SumDoubleAgg result = new SumDoubleAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            SumDoubleAgg myagg = (SumDoubleAgg) agg;
-            myagg.empty = true;
-            myagg.sum = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			SumDoubleAgg myagg = (SumDoubleAgg) agg;
+			myagg.empty = true;
+			myagg.sum = 0;
+		}
 
-        boolean warned = false;
+		boolean warned = false;
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            try {
-                merge(agg, parameters[0]);
-            } catch (NumberFormatException e) {
-                if (!warned) {
-                    warned = true;
-                    LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
-                    LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
-                }
-            }
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			try {
+				merge(agg, parameters[0]);
+			} catch (NumberFormatException e) {
+				if (!warned) {
+					warned = true;
+					LOG.warn(getClass().getSimpleName() + " "
+							+ StringUtils.stringifyException(e));
+					LOG.warn(getClass().getSimpleName()
+							+ " ignoring similar exceptions.");
+				}
+			}
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            return terminate(agg);
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			return terminate(agg);
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                SumDoubleAgg myagg = (SumDoubleAgg) agg;
-                myagg.empty = false;
-                myagg.sum += PrimitiveObjectInspectorUtils.getDouble(partial, inputOI);
-            }
-        }
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				SumDoubleAgg myagg = (SumDoubleAgg) agg;
+				myagg.empty = false;
+				myagg.sum += PrimitiveObjectInspectorUtils.getDouble(partial,
+						inputOI);
+			}
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            SumDoubleAgg myagg = (SumDoubleAgg) agg;
-            if (myagg.empty) {
-                return null;
-            }
-            result.set(myagg.sum);
-            return result;
-        }
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			SumDoubleAgg myagg = (SumDoubleAgg) agg;
+			if (myagg.empty) {
+				return null;
+			}
+			result.set(myagg.sum);
+			return result;
+		}
 
-    }
+	}
 
-    /**
-     * GenericUDAFSumLong.
-     * 
-     */
-    public static class GenericUDAFSumLong extends GenericUDAFEvaluator {
-        private PrimitiveObjectInspector inputOI;
-        private LongWritable result;
+	/**
+	 * GenericUDAFSumLong.
+	 * 
+	 */
+	public static class GenericUDAFSumLong extends GenericUDAFEvaluator {
+		private PrimitiveObjectInspector inputOI;
+		private LongWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            super.init(m, parameters);
-            result = new LongWritable(0);
-            inputOI = (PrimitiveObjectInspector) parameters[0];
-            return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-        }
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			super.init(m, parameters);
+			result = new LongWritable(0);
+			inputOI = (PrimitiveObjectInspector) parameters[0];
+			return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+		}
 
-        /** class for storing double sum value. */
-        static class SumLongAgg implements SerializableBuffer {
-            boolean empty;
-            long sum;
+		/** class for storing double sum value. */
+		static class SumLongAgg implements SerializableBuffer {
+			boolean empty;
+			long sum;
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                empty = BufferSerDeUtil.getBoolean(data, start);
-                start += 1;
-                sum = BufferSerDeUtil.getLong(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				empty = BufferSerDeUtil.getBoolean(data, start);
+				start += 1;
+				sum = BufferSerDeUtil.getLong(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeBoolean(empty, data, start);
-                start += 1;
-                BufferSerDeUtil.writeLong(sum, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeBoolean(empty, data, start);
+				start += 1;
+				BufferSerDeUtil.writeLong(sum, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeBoolean(empty);
-                output.writeLong(sum);
-            }
-        }
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeBoolean(empty);
+				output.writeLong(sum);
+			}
+		}
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            SumLongAgg result = new SumLongAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			SumLongAgg result = new SumLongAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            SumLongAgg myagg = (SumLongAgg) agg;
-            myagg.empty = true;
-            myagg.sum = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			SumLongAgg myagg = (SumLongAgg) agg;
+			myagg.empty = true;
+			myagg.sum = 0;
+		}
 
-        private boolean warned = false;
+		private boolean warned = false;
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            try {
-                merge(agg, parameters[0]);
-            } catch (NumberFormatException e) {
-                if (!warned) {
-                    warned = true;
-                    LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
-                }
-            }
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			try {
+				merge(agg, parameters[0]);
+			} catch (NumberFormatException e) {
+				if (!warned) {
+					warned = true;
+					LOG.warn(getClass().getSimpleName() + " "
+							+ StringUtils.stringifyException(e));
+				}
+			}
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            return terminate(agg);
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			return terminate(agg);
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                SumLongAgg myagg = (SumLongAgg) agg;
-                myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial, inputOI);
-                myagg.empty = false;
-            }
-        }
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				SumLongAgg myagg = (SumLongAgg) agg;
+				myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial,
+						inputOI);
+				myagg.empty = false;
+			}
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            SumLongAgg myagg = (SumLongAgg) agg;
-            if (myagg.empty) {
-                return null;
-            }
-            result.set(myagg.sum);
-            return result;
-        }
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			SumLongAgg myagg = (SumLongAgg) agg;
+			if (myagg.empty) {
+				return null;
+			}
+			result.set(myagg.sum);
+			return result;
+		}
 
-    }
+	}
 
 }
diff --git a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
index d91551d..4c16f5a 100644
--- a/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
+++ b/hivesterix/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
@@ -53,260 +53,279 @@
 @Description(name = "variance,var_pop", value = "_FUNC_(x) - Returns the variance of a set of numbers")
 public class GenericUDAFVariance extends AbstractGenericUDAFResolver {
 
-    static final Log LOG = LogFactory.getLog(GenericUDAFVariance.class.getName());
+	static final Log LOG = LogFactory.getLog(GenericUDAFVariance.class
+			.getName());
 
-    @Override
-    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-        if (parameters.length != 1) {
-            throw new UDFArgumentTypeException(parameters.length - 1, "Exactly one argument is expected.");
-        }
+	@Override
+	public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+			throws SemanticException {
+		if (parameters.length != 1) {
+			throw new UDFArgumentTypeException(parameters.length - 1,
+					"Exactly one argument is expected.");
+		}
 
-        if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
-                    + parameters[0].getTypeName() + " is passed.");
-        }
-        switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
-            case BYTE:
-            case SHORT:
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-            case STRING:
-                return new GenericUDAFVarianceEvaluator();
-            case BOOLEAN:
-            default:
-                throw new UDFArgumentTypeException(0, "Only numeric or string type arguments are accepted but "
-                        + parameters[0].getTypeName() + " is passed.");
-        }
-    }
+		if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+			throw new UDFArgumentTypeException(0,
+					"Only primitive type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+		switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+		case BYTE:
+		case SHORT:
+		case INT:
+		case LONG:
+		case FLOAT:
+		case DOUBLE:
+		case STRING:
+			return new GenericUDAFVarianceEvaluator();
+		case BOOLEAN:
+		default:
+			throw new UDFArgumentTypeException(0,
+					"Only numeric or string type arguments are accepted but "
+							+ parameters[0].getTypeName() + " is passed.");
+		}
+	}
 
-    /**
-     * Evaluate the variance using the algorithm described by Chan, Golub, and
-     * LeVeque in
-     * "Algorithms for computing the sample variance: analysis and recommendations"
-     * The American Statistician, 37 (1983) pp. 242--247.
-     * 
-     * variance = variance1 + variance2 + n/(m*(m+n)) * pow(((m/n)*t1 - t2),2)
-     * 
-     * where: - variance is sum[x-avg^2] (this is actually n times the variance)
-     * and is updated at every step. - n is the count of elements in chunk1 - m
-     * is the count of elements in chunk2 - t1 = sum of elements in chunk1, t2 =
-     * sum of elements in chunk2.
-     * 
-     * This algorithm was proven to be numerically stable by J.L. Barlow in
-     * "Error analysis of a pairwise summation algorithm to compute sample variance"
-     * Numer. Math, 58 (1991) pp. 583--590
-     * 
-     */
-    public static class GenericUDAFVarianceEvaluator extends GenericUDAFEvaluator {
+	/**
+	 * Evaluate the variance using the algorithm described by Chan, Golub, and
+	 * LeVeque in
+	 * "Algorithms for computing the sample variance: analysis and recommendations"
+	 * The American Statistician, 37 (1983) pp. 242--247.
+	 * 
+	 * variance = variance1 + variance2 + n/(m*(m+n)) * pow(((m/n)*t1 - t2),2)
+	 * 
+	 * where: - variance is sum[x-avg^2] (this is actually n times the variance)
+	 * and is updated at every step. - n is the count of elements in chunk1 - m
+	 * is the count of elements in chunk2 - t1 = sum of elements in chunk1, t2 =
+	 * sum of elements in chunk2.
+	 * 
+	 * This algorithm was proven to be numerically stable by J.L. Barlow in
+	 * "Error analysis of a pairwise summation algorithm to compute sample variance"
+	 * Numer. Math, 58 (1991) pp. 583--590
+	 * 
+	 */
+	public static class GenericUDAFVarianceEvaluator extends
+			GenericUDAFEvaluator {
 
-        // For PARTIAL1 and COMPLETE
-        private PrimitiveObjectInspector inputOI;
+		// For PARTIAL1 and COMPLETE
+		private PrimitiveObjectInspector inputOI;
 
-        // For PARTIAL2 and FINAL
-        private StructObjectInspector soi;
-        private StructField countField;
-        private StructField sumField;
-        private StructField varianceField;
-        private LongObjectInspector countFieldOI;
-        private DoubleObjectInspector sumFieldOI;
-        private DoubleObjectInspector varianceFieldOI;
+		// For PARTIAL2 and FINAL
+		private StructObjectInspector soi;
+		private StructField countField;
+		private StructField sumField;
+		private StructField varianceField;
+		private LongObjectInspector countFieldOI;
+		private DoubleObjectInspector sumFieldOI;
 
-        // For PARTIAL1 and PARTIAL2
-        private Object[] partialResult;
+		// For PARTIAL1 and PARTIAL2
+		private Object[] partialResult;
 
-        // For FINAL and COMPLETE
-        private DoubleWritable result;
+		// For FINAL and COMPLETE
+		private DoubleWritable result;
 
-        @Override
-        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            super.init(m, parameters);
+		@Override
+		public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			super.init(m, parameters);
 
-            // init input
-            if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-                inputOI = (PrimitiveObjectInspector) parameters[0];
-            } else {
-                soi = (StructObjectInspector) parameters[0];
+			// init input
+			if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+				inputOI = (PrimitiveObjectInspector) parameters[0];
+			} else {
+				soi = (StructObjectInspector) parameters[0];
 
-                countField = soi.getStructFieldRef("count");
-                sumField = soi.getStructFieldRef("sum");
-                varianceField = soi.getStructFieldRef("variance");
+				countField = soi.getStructFieldRef("count");
+				sumField = soi.getStructFieldRef("sum");
+				varianceField = soi.getStructFieldRef("variance");
 
-                countFieldOI = (LongObjectInspector) countField.getFieldObjectInspector();
-                sumFieldOI = (DoubleObjectInspector) sumField.getFieldObjectInspector();
-                varianceFieldOI = (DoubleObjectInspector) varianceField.getFieldObjectInspector();
-            }
+				countFieldOI = (LongObjectInspector) countField
+						.getFieldObjectInspector();
+				sumFieldOI = (DoubleObjectInspector) sumField
+						.getFieldObjectInspector();
+			}
 
-            // init output
-            if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-                // The output of a partial aggregation is a struct containing
-                // a long count and doubles sum and variance.
+			// init output
+			if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+				// The output of a partial aggregation is a struct containing
+				// a long count and doubles sum and variance.
 
-                ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
+				ArrayList<ObjectInspector> foi = new ArrayList<ObjectInspector>();
 
-                foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-                foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+				foi.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
 
-                ArrayList<String> fname = new ArrayList<String>();
-                fname.add("count");
-                fname.add("sum");
-                fname.add("variance");
+				ArrayList<String> fname = new ArrayList<String>();
+				fname.add("count");
+				fname.add("sum");
+				fname.add("variance");
 
-                partialResult = new Object[3];
-                partialResult[0] = new LongWritable(0);
-                partialResult[1] = new DoubleWritable(0);
-                partialResult[2] = new DoubleWritable(0);
+				partialResult = new Object[3];
+				partialResult[0] = new LongWritable(0);
+				partialResult[1] = new DoubleWritable(0);
+				partialResult[2] = new DoubleWritable(0);
 
-                return ObjectInspectorFactory.getStandardStructObjectInspector(fname, foi);
+				return ObjectInspectorFactory.getStandardStructObjectInspector(
+						fname, foi);
 
-            } else {
-                setResult(new DoubleWritable(0));
-                return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
-            }
-        }
+			} else {
+				setResult(new DoubleWritable(0));
+				return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+			}
+		}
 
-        static class StdAgg implements SerializableBuffer {
-            long count; // number of elements
-            double sum; // sum of elements
-            double variance; // sum[x-avg^2] (this is actually n times the
-                             // variance)
+		static class StdAgg implements SerializableBuffer {
+			long count; // number of elements
+			double sum; // sum of elements
+			double variance; // sum[x-avg^2] (this is actually n times the
+								// variance)
 
-            @Override
-            public void deSerializeAggBuffer(byte[] data, int start, int len) {
-                count = BufferSerDeUtil.getLong(data, start);
-                start += 8;
-                sum = BufferSerDeUtil.getDouble(data, start);
-                start += 8;
-                variance = BufferSerDeUtil.getDouble(data, start);
-            }
+			@Override
+			public void deSerializeAggBuffer(byte[] data, int start, int len) {
+				count = BufferSerDeUtil.getLong(data, start);
+				start += 8;
+				sum = BufferSerDeUtil.getDouble(data, start);
+				start += 8;
+				variance = BufferSerDeUtil.getDouble(data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(byte[] data, int start, int len) {
-                BufferSerDeUtil.writeLong(count, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(sum, data, start);
-                start += 8;
-                BufferSerDeUtil.writeDouble(variance, data, start);
-            }
+			@Override
+			public void serializeAggBuffer(byte[] data, int start, int len) {
+				BufferSerDeUtil.writeLong(count, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(sum, data, start);
+				start += 8;
+				BufferSerDeUtil.writeDouble(variance, data, start);
+			}
 
-            @Override
-            public void serializeAggBuffer(DataOutput output) throws IOException {
-                output.writeLong(count);
-                output.writeDouble(sum);
-                output.writeDouble(variance);
-            }
-        };
+			@Override
+			public void serializeAggBuffer(DataOutput output)
+					throws IOException {
+				output.writeLong(count);
+				output.writeDouble(sum);
+				output.writeDouble(variance);
+			}
+		};
 
-        @Override
-        public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-            StdAgg result = new StdAgg();
-            reset(result);
-            return result;
-        }
+		@Override
+		public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+			StdAgg result = new StdAgg();
+			reset(result);
+			return result;
+		}
 
-        @Override
-        public void reset(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            myagg.count = 0;
-            myagg.sum = 0;
-            myagg.variance = 0;
-        }
+		@Override
+		public void reset(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			myagg.count = 0;
+			myagg.sum = 0;
+			myagg.variance = 0;
+		}
 
-        private boolean warned = false;
+		private boolean warned = false;
 
-        @Override
-        public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-            assert (parameters.length == 1);
-            Object p = parameters[0];
-            if (p != null) {
-                StdAgg myagg = (StdAgg) agg;
-                try {
-                    double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
-                    myagg.count++;
-                    myagg.sum += v;
-                    if (myagg.count > 1) {
-                        double t = myagg.count * v - myagg.sum;
-                        myagg.variance += (t * t) / ((double) myagg.count * (myagg.count - 1));
-                    }
-                } catch (NumberFormatException e) {
-                    if (!warned) {
-                        warned = true;
-                        LOG.warn(getClass().getSimpleName() + " " + StringUtils.stringifyException(e));
-                        LOG.warn(getClass().getSimpleName() + " ignoring similar exceptions.");
-                    }
-                }
-            }
-        }
+		@Override
+		public void iterate(AggregationBuffer agg, Object[] parameters)
+				throws HiveException {
+			assert (parameters.length == 1);
+			Object p = parameters[0];
+			if (p != null) {
+				StdAgg myagg = (StdAgg) agg;
+				try {
+					double v = PrimitiveObjectInspectorUtils.getDouble(p,
+							inputOI);
+					myagg.count++;
+					myagg.sum += v;
+					if (myagg.count > 1) {
+						double t = myagg.count * v - myagg.sum;
+						myagg.variance += (t * t)
+								/ ((double) myagg.count * (myagg.count - 1));
+					}
+				} catch (NumberFormatException e) {
+					if (!warned) {
+						warned = true;
+						LOG.warn(getClass().getSimpleName() + " "
+								+ StringUtils.stringifyException(e));
+						LOG.warn(getClass().getSimpleName()
+								+ " ignoring similar exceptions.");
+					}
+				}
+			}
+		}
 
-        @Override
-        public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
-            ((LongWritable) partialResult[0]).set(myagg.count);
-            ((DoubleWritable) partialResult[1]).set(myagg.sum);
-            ((DoubleWritable) partialResult[2]).set(myagg.variance);
-            return partialResult;
-        }
+		@Override
+		public Object terminatePartial(AggregationBuffer agg)
+				throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
+			((LongWritable) partialResult[0]).set(myagg.count);
+			((DoubleWritable) partialResult[1]).set(myagg.sum);
+			((DoubleWritable) partialResult[2]).set(myagg.variance);
+			return partialResult;
+		}
 
-        @Override
-        public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-            if (partial != null) {
-                StdAgg myagg = (StdAgg) agg;
+		@Override
+		public void merge(AggregationBuffer agg, Object partial)
+				throws HiveException {
+			if (partial != null) {
+				StdAgg myagg = (StdAgg) agg;
 
-                Object partialCount = soi.getStructFieldData(partial, countField);
-                Object partialSum = soi.getStructFieldData(partial, sumField);
-                Object partialVariance = soi.getStructFieldData(partial, varianceField);
+				Object partialCount = soi.getStructFieldData(partial,
+						countField);
+				Object partialSum = soi.getStructFieldData(partial, sumField);
+				Object partialVariance = soi.getStructFieldData(partial,
+						varianceField);
 
-                long n = myagg.count;
-                long m = countFieldOI.get(partialCount);
+				long n = myagg.count;
+				long m = countFieldOI.get(partialCount);
 
-                if (n == 0) {
-                    // Just copy the information since there is nothing so far
-                    myagg.variance = sumFieldOI.get(partialVariance);
-                    myagg.count = countFieldOI.get(partialCount);
-                    myagg.sum = sumFieldOI.get(partialSum);
-                }
+				if (n == 0) {
+					// Just copy the information since there is nothing so far
+					myagg.variance = sumFieldOI.get(partialVariance);
+					myagg.count = countFieldOI.get(partialCount);
+					myagg.sum = sumFieldOI.get(partialSum);
+				}
 
-                if (m != 0 && n != 0) {
-                    // Merge the two partials
+				if (m != 0 && n != 0) {
+					// Merge the two partials
 
-                    double a = myagg.sum;
-                    double b = sumFieldOI.get(partialSum);
+					double a = myagg.sum;
+					double b = sumFieldOI.get(partialSum);
 
-                    myagg.count += m;
-                    myagg.sum += b;
-                    double t = (m / (double) n) * a - b;
-                    myagg.variance += sumFieldOI.get(partialVariance) + ((n / (double) m) / ((double) n + m)) * t * t;
-                }
-            }
-        }
+					myagg.count += m;
+					myagg.sum += b;
+					double t = (m / (double) n) * a - b;
+					myagg.variance += sumFieldOI.get(partialVariance)
+							+ ((n / (double) m) / ((double) n + m)) * t * t;
+				}
+			}
+		}
 
-        @Override
-        public Object terminate(AggregationBuffer agg) throws HiveException {
-            StdAgg myagg = (StdAgg) agg;
+		@Override
+		public Object terminate(AggregationBuffer agg) throws HiveException {
+			StdAgg myagg = (StdAgg) agg;
 
-            if (myagg.count == 0) { // SQL standard - return null for zero
-                                    // elements
-                return null;
-            } else {
-                if (myagg.count > 1) {
-                    getResult().set(myagg.variance / (myagg.count));
-                } else { // for one element the variance is always 0
-                    getResult().set(0);
-                }
-                return getResult();
-            }
-        }
+			if (myagg.count == 0) { // SQL standard - return null for zero
+									// elements
+				return null;
+			} else {
+				if (myagg.count > 1) {
+					getResult().set(myagg.variance / (myagg.count));
+				} else { // for one element the variance is always 0
+					getResult().set(0);
+				}
+				return getResult();
+			}
+		}
 
-        public void setResult(DoubleWritable result) {
-            this.result = result;
-        }
+		public void setResult(DoubleWritable result) {
+			this.result = result;
+		}
 
-        public DoubleWritable getResult() {
-            return result;
-        }
-    }
+		public DoubleWritable getResult() {
+			return result;
+		}
+	}
 
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestCase.java
index 3a0f381..a69a3f2 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestCase.java
@@ -15,128 +15,130 @@
 import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
 
 public class PerfTestCase extends AbstractPerfTestCase {
-    private File resultFile;
-    private FileSystem dfs;
+	private File resultFile;
+	private FileSystem dfs;
 
-    PerfTestCase(File queryFile, File resultFile) {
-        super("testRuntimeFunction", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	PerfTestCase(File queryFile, File resultFile) {
+		super("testRuntimeFunction", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testRuntimeFunction() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	@Test
+	public void testRuntimeFunction() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
 
-        dfs = FileSystem.get(ConfUtil.getJobConf());
+		dfs = FileSystem.get(ConfUtil.getJobConf());
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            driver.run(query);
-            driver.clear();
-            i++;
-        }
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			driver.run(query);
+			driver.clear();
+			i++;
+		}
 
-        String warehouse = hconf.get("hive.metastore.warehouse.dir");
-        String tableName = removeExt(resultFile.getName());
-        String directory = warehouse + "/" + tableName + "/";
-        String localDirectory = "tmp";
+		String warehouse = hconf.get("hive.metastore.warehouse.dir");
+		String tableName = removeExt(resultFile.getName());
+		String directory = warehouse + "/" + tableName + "/";
+		String localDirectory = "tmp";
 
-        FileStatus[] files = dfs.listStatus(new Path(directory));
-        FileSystem lfs = null;
-        if (files == null) {
-            lfs = FileSystem.getLocal(ConfUtil.getJobConf());
-            files = lfs.listStatus(new Path(directory));
-        }
+		FileStatus[] files = dfs.listStatus(new Path(directory));
+		FileSystem lfs = null;
+		if (files == null) {
+			lfs = FileSystem.getLocal(ConfUtil.getJobConf());
+			files = lfs.listStatus(new Path(directory));
+		}
 
-        File resultDirectory = new File(localDirectory + "/" + tableName);
-        deleteDir(resultDirectory);
-        resultDirectory.mkdir();
+		File resultDirectory = new File(localDirectory + "/" + tableName);
+		deleteDir(resultDirectory);
+		resultDirectory.mkdir();
 
-        for (FileStatus fs : files) {
-            Path src = fs.getPath();
-            if (src.getName().indexOf("crc") >= 0)
-                continue;
+		for (FileStatus fs : files) {
+			Path src = fs.getPath();
+			if (src.getName().indexOf("crc") >= 0)
+				continue;
 
-            String destStr = localDirectory + "/" + tableName + "/" + src.getName();
-            Path dest = new Path(destStr);
-            if (lfs != null) {
-                lfs.copyToLocalFile(src, dest);
-                dfs.copyFromLocalFile(dest, new Path(directory));
-            } else
-                dfs.copyToLocalFile(src, dest);
-        }
+			String destStr = localDirectory + "/" + tableName + "/"
+					+ src.getName();
+			Path dest = new Path(destStr);
+			if (lfs != null) {
+				lfs.copyToLocalFile(src, dest);
+				dfs.copyFromLocalFile(dest, new Path(directory));
+			} else
+				dfs.copyToLocalFile(src, dest);
+		}
 
-        File[] rFiles = resultDirectory.listFiles();
-        StringBuilder sb = new StringBuilder();
-        for (File r : rFiles) {
-            if (r.getName().indexOf("crc") >= 0)
-                continue;
-            readFileToString(r, sb);
-        }
-        deleteDir(resultDirectory);
+		File[] rFiles = resultDirectory.listFiles();
+		StringBuilder sb = new StringBuilder();
+		for (File r : rFiles) {
+			if (r.getName().indexOf("crc") >= 0)
+				continue;
+			readFileToString(r, sb);
+		}
+		deleteDir(resultDirectory);
 
-        StringBuilder buf = new StringBuilder();
-        readFileToString(resultFile, buf);
-        if (!equal(buf, sb)) {
-            throw new Exception("Result for " + queryFile + " changed:\n" + sw.toString());
-        }
-    }
+		StringBuilder buf = new StringBuilder();
+		readFileToString(resultFile, buf);
+		if (!equal(buf, sb)) {
+			throw new Exception("Result for " + queryFile + " changed:\n"
+					+ sw.toString());
+		}
+	}
 
-    private void deleteDir(File resultDirectory) {
-        if (resultDirectory.exists()) {
-            File[] rFiles = resultDirectory.listFiles();
-            for (File r : rFiles)
-                r.delete();
-            resultDirectory.delete();
-        }
-    }
+	private void deleteDir(File resultDirectory) {
+		if (resultDirectory.exists()) {
+			File[] rFiles = resultDirectory.listFiles();
+			for (File r : rFiles)
+				r.delete();
+			resultDirectory.delete();
+		}
+	}
 
-    private boolean equal(StringBuilder sb1, StringBuilder sb2) {
-        String s1 = sb1.toString();
-        String s2 = sb2.toString();
-        String[] rowsOne = s1.split("\n");
-        String[] rowsTwo = s2.split("\n");
+	private boolean equal(StringBuilder sb1, StringBuilder sb2) {
+		String s1 = sb1.toString();
+		String s2 = sb2.toString();
+		String[] rowsOne = s1.split("\n");
+		String[] rowsTwo = s2.split("\n");
 
-        if (rowsOne.length != rowsTwo.length)
-            return false;
+		if (rowsOne.length != rowsTwo.length)
+			return false;
 
-        for (int i = 0; i < rowsOne.length; i++) {
-            String row1 = rowsOne[i];
-            String row2 = rowsTwo[i];
+		for (int i = 0; i < rowsOne.length; i++) {
+			String row1 = rowsOne[i];
+			String row2 = rowsTwo[i];
 
-            if (row1.equals(row2))
-                continue;
+			if (row1.equals(row2))
+				continue;
 
-            String[] fields1 = row1.split("");
-            String[] fields2 = row2.split("");
+			String[] fields1 = row1.split("");
+			String[] fields2 = row2.split("");
 
-            for (int j = 0; j < fields1.length; j++) {
-                if (fields1[j].equals(fields2[j])) {
-                    continue;
-                } else if (fields1[j].indexOf('.') < 0) {
-                    return false;
-                } else {
-                    Float float1 = Float.parseFloat(fields1[j]);
-                    Float float2 = Float.parseFloat(fields2[j]);
+			for (int j = 0; j < fields1.length; j++) {
+				if (fields1[j].equals(fields2[j])) {
+					continue;
+				} else if (fields1[j].indexOf('.') < 0) {
+					return false;
+				} else {
+					Float float1 = Float.parseFloat(fields1[j]);
+					Float float2 = Float.parseFloat(fields2[j]);
 
-                    if (Math.abs(float1 - float2) == 0)
-                        continue;
-                    else
-                        return false;
-                }
-            }
-        }
+					if (Math.abs(float1 - float2) == 0)
+						continue;
+					else
+						return false;
+				}
+			}
+		}
 
-        return true;
-    }
+		return true;
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuite.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuite.java
index 796842d..8d8178f 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuite.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuite.java
@@ -9,66 +9,66 @@
 

 public class PerfTestSuite extends AbstractPerfTestSuiteClass {

 

-    private static final String PATH_TO_QUERIES = "src/test/resources/perf/queries/";

-    private static final String PATH_TO_RESULTS = "src/test/resources/perf/results/";

-    private static final String PATH_TO_IGNORES = "src/test/resources/perf/ignore.txt";

+	private static final String PATH_TO_QUERIES = "src/test/resources/perf/queries/";

+	private static final String PATH_TO_RESULTS = "src/test/resources/perf/results/";

+	private static final String PATH_TO_IGNORES = "src/test/resources/perf/ignore.txt";

 

-    private static final String FILE_EXTENSION_OF_RESULTS = "result";

+	private static final String FILE_EXTENSION_OF_RESULTS = "result";

 

-    public static Test suite() throws Exception {

-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

-        File testData = new File(PATH_TO_QUERIES);

-        File[] queries = testData.listFiles();

-        PerfTestSuite testSuite = new PerfTestSuite();

+	public static Test suite() throws Exception {

+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

+		File testData = new File(PATH_TO_QUERIES);

+		File[] queries = testData.listFiles();

+		PerfTestSuite testSuite = new PerfTestSuite();

 

-        // set hdfs and hyracks cluster, and load test data to hdfs

-        try {

-            testSuite.setup();

-            testSuite.loadData();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

+		// set hdfs and hyracks cluster, and load test data to hdfs

+		try {

+			testSuite.setup();

+			testSuite.loadData();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

 

-        for (File qFile : queries) {

-            if (isIgnored(qFile.getName(), ignores))

-                continue;

+		for (File qFile : queries) {

+			if (isIgnored(qFile.getName(), ignores))

+				continue;

 

-            if (qFile.isFile()) {

-                String resultFileName = hiveExtToResExt(qFile.getName());

-                File rFile = new File(PATH_TO_RESULTS + resultFileName);

-                testSuite.addTest(new PerfTestCase(qFile, rFile));

-            }

-        }

-        return testSuite;

-    }

+			if (qFile.isFile()) {

+				String resultFileName = hiveExtToResExt(qFile.getName());

+				File rFile = new File(PATH_TO_RESULTS + resultFileName);

+				testSuite.addTest(new PerfTestCase(qFile, rFile));

+			}

+		}

+		return testSuite;

+	}

 

-    private static String hiveExtToResExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

-    }

+	private static String hiveExtToResExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

+	}

 

-    /**

-     * Runs the tests and collects their result in a TestResult.

-     */

-    @Override

-    public void run(TestResult result) {

+	/**

+	 * Runs the tests and collects their result in a TestResult.

+	 */

+	@Override

+	public void run(TestResult result) {

 

-        int testCount = countTestCases();

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

-            Test each = this.testAt(i);

-            if (result.shouldStop())

-                break;

-            runTest(each, result);

-        }

+		int testCount = countTestCases();

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

+			Test each = this.testAt(i);

+			if (result.shouldStop())

+				break;

+			runTest(each, result);

+		}

 

-        // cleanup hdfs and hyracks cluster

-        try {

-            cleanup();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

-    }

+		// cleanup hdfs and hyracks cluster

+		try {

+			cleanup();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteCaseGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteCaseGenerator.java
index 75a325e..258db22 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteCaseGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteCaseGenerator.java
@@ -13,87 +13,89 @@
 import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
 
 public class PerfTestSuiteCaseGenerator extends AbstractPerfTestCase {
-    private File resultFile;
-    private FileSystem dfs;
+	private File resultFile;
+	private FileSystem dfs;
 
-    PerfTestSuiteCaseGenerator(File queryFile, File resultFile) {
-        super("testRuntimeFunction", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	PerfTestSuiteCaseGenerator(File queryFile, File resultFile) {
+		super("testRuntimeFunction", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testRuntimeFunction() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
+	@Test
+	public void testRuntimeFunction() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf);
-        driver.init();
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf);
+		driver.init();
 
-        dfs = FileSystem.get(ConfUtil.getJobConf());
+		dfs = FileSystem.get(ConfUtil.getJobConf());
 
-        long startTime = System.currentTimeMillis();
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            driver.run(query);
-            // driver.clear();
-            i++;
-        }
-        long endTime = System.currentTimeMillis();
-        System.out.println(resultFile.getName() + " execution time " + (endTime - startTime));
+		long startTime = System.currentTimeMillis();
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			driver.run(query);
+			// driver.clear();
+			i++;
+		}
+		long endTime = System.currentTimeMillis();
+		System.out.println(resultFile.getName() + " execution time "
+				+ (endTime - startTime));
 
-        String warehouse = hconf.get("hive.metastore.warehouse.dir");
-        String tableName = removeExt(resultFile.getName());
-        String directory = warehouse + "/" + tableName + "/";
-        String localDirectory = "tmp";
+		String warehouse = hconf.get("hive.metastore.warehouse.dir");
+		String tableName = removeExt(resultFile.getName());
+		String directory = warehouse + "/" + tableName + "/";
+		String localDirectory = "tmp";
 
-        FileStatus[] files = dfs.listStatus(new Path(directory));
-        FileSystem lfs = null;
-        if (files == null) {
-            lfs = FileSystem.getLocal(ConfUtil.getJobConf());
-            files = lfs.listStatus(new Path(directory));
-        }
+		FileStatus[] files = dfs.listStatus(new Path(directory));
+		FileSystem lfs = null;
+		if (files == null) {
+			lfs = FileSystem.getLocal(ConfUtil.getJobConf());
+			files = lfs.listStatus(new Path(directory));
+		}
 
-        File resultDirectory = new File(localDirectory + "/" + tableName);
-        deleteDir(resultDirectory);
-        resultDirectory.mkdir();
+		File resultDirectory = new File(localDirectory + "/" + tableName);
+		deleteDir(resultDirectory);
+		resultDirectory.mkdir();
 
-        for (FileStatus fs : files) {
-            Path src = fs.getPath();
-            if (src.getName().indexOf("crc") >= 0)
-                continue;
+		for (FileStatus fs : files) {
+			Path src = fs.getPath();
+			if (src.getName().indexOf("crc") >= 0)
+				continue;
 
-            String destStr = localDirectory + "/" + tableName + "/" + src.getName();
-            Path dest = new Path(destStr);
-            if (lfs != null) {
-                lfs.copyToLocalFile(src, dest);
-                dfs.copyFromLocalFile(dest, new Path(directory));
-            } else
-                dfs.copyToLocalFile(src, dest);
-        }
+			String destStr = localDirectory + "/" + tableName + "/"
+					+ src.getName();
+			Path dest = new Path(destStr);
+			if (lfs != null) {
+				lfs.copyToLocalFile(src, dest);
+				dfs.copyFromLocalFile(dest, new Path(directory));
+			} else
+				dfs.copyToLocalFile(src, dest);
+		}
 
-        File[] rFiles = resultDirectory.listFiles();
-        StringBuilder sb = new StringBuilder();
-        for (File r : rFiles) {
-            if (r.getName().indexOf("crc") >= 0)
-                continue;
-            readFileToString(r, sb);
-        }
-        deleteDir(resultDirectory);
+		File[] rFiles = resultDirectory.listFiles();
+		StringBuilder sb = new StringBuilder();
+		for (File r : rFiles) {
+			if (r.getName().indexOf("crc") >= 0)
+				continue;
+			readFileToString(r, sb);
+		}
+		deleteDir(resultDirectory);
 
-        writeStringToFile(resultFile, sb);
-    }
+		writeStringToFile(resultFile, sb);
+	}
 
-    private void deleteDir(File resultDirectory) {
-        if (resultDirectory.exists()) {
-            File[] rFiles = resultDirectory.listFiles();
-            for (File r : rFiles)
-                r.delete();
-            resultDirectory.delete();
-        }
-    }
+	private void deleteDir(File resultDirectory) {
+		if (resultDirectory.exists()) {
+			File[] rFiles = resultDirectory.listFiles();
+			for (File r : rFiles)
+				r.delete();
+			resultDirectory.delete();
+		}
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteGenerator.java
index aa38014..0a27ca2 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/PerfTestSuiteGenerator.java
@@ -9,66 +9,66 @@
 

 public class PerfTestSuiteGenerator extends AbstractPerfTestSuiteClass {

 

-    private static final String PATH_TO_QUERIES = "src/test/resources/perf/queries/";

-    private static final String PATH_TO_RESULTS = "src/test/resources/perf/results/";

-    private static final String PATH_TO_IGNORES = "src/test/resources/perf/ignore.txt";

+	private static final String PATH_TO_QUERIES = "src/test/resources/perf/queries/";

+	private static final String PATH_TO_RESULTS = "src/test/resources/perf/results/";

+	private static final String PATH_TO_IGNORES = "src/test/resources/perf/ignore.txt";

 

-    private static final String FILE_EXTENSION_OF_RESULTS = "result";

+	private static final String FILE_EXTENSION_OF_RESULTS = "result";

 

-    public static Test suite() throws Exception {

-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

-        File testData = new File(PATH_TO_QUERIES);

-        File[] queries = testData.listFiles();

-        PerfTestSuiteGenerator testSuite = new PerfTestSuiteGenerator();

+	public static Test suite() throws Exception {

+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

+		File testData = new File(PATH_TO_QUERIES);

+		File[] queries = testData.listFiles();

+		PerfTestSuiteGenerator testSuite = new PerfTestSuiteGenerator();

 

-        // set hdfs and hyracks cluster, and load test data to hdfs

-        try {

-            testSuite.setup();

-            testSuite.loadData();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

+		// set hdfs and hyracks cluster, and load test data to hdfs

+		try {

+			testSuite.setup();

+			testSuite.loadData();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

 

-        for (File qFile : queries) {

-            if (isIgnored(qFile.getName(), ignores))

-                continue;

+		for (File qFile : queries) {

+			if (isIgnored(qFile.getName(), ignores))

+				continue;

 

-            if (qFile.isFile() && qFile.getName().startsWith("q18_")) {

-                String resultFileName = hiveExtToResExt(qFile.getName());

-                File rFile = new File(PATH_TO_RESULTS + resultFileName);

-                testSuite.addTest(new PerfTestSuiteCaseGenerator(qFile, rFile));

-            }

-        }

-        return testSuite;

-    }

+			if (qFile.isFile() && qFile.getName().startsWith("q18_")) {

+				String resultFileName = hiveExtToResExt(qFile.getName());

+				File rFile = new File(PATH_TO_RESULTS + resultFileName);

+				testSuite.addTest(new PerfTestSuiteCaseGenerator(qFile, rFile));

+			}

+		}

+		return testSuite;

+	}

 

-    private static String hiveExtToResExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

-    }

+	private static String hiveExtToResExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

+	}

 

-    /**

-     * Runs the tests and collects their result in a TestResult.

-     */

-    @Override

-    public void run(TestResult result) {

+	/**

+	 * Runs the tests and collects their result in a TestResult.

+	 */

+	@Override

+	public void run(TestResult result) {

 

-        int testCount = countTestCases();

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

-            Test each = this.testAt(i);

-            if (result.shouldStop())

-                break;

-            runTest(each, result);

-        }

+		int testCount = countTestCases();

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

+			Test each = this.testAt(i);

+			if (result.shouldStop())

+				break;

+			runTest(each, result);

+		}

 

-        // cleanup hdfs and hyracks cluster

-        try {

-            cleanup();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

-    }

+		// cleanup hdfs and hyracks cluster

+		try {

+			cleanup();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestCase.java
index 7e7db36..f55d6a1 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestCase.java
@@ -10,40 +10,43 @@
 import junit.framework.TestCase;

 

 public class AbstractPerfTestCase extends TestCase {

-    protected File queryFile;

+	protected File queryFile;

 

-    public AbstractPerfTestCase(String testName, File queryFile) {

-        super(testName);

-    }

+	public AbstractPerfTestCase(String testName, File queryFile) {

+		super(testName);

+	}

 

-    protected static void readFileToString(File file, StringBuilder buf) throws Exception {

-        BufferedReader result = new BufferedReader(new FileReader(file));

-        while (true) {

-            String s = result.readLine();

-            if (s == null) {

-                break;

-            } else {

-                buf.append(s);

-                buf.append('\n');

-            }

-        }

-        result.close();

-    }

+	protected static void readFileToString(File file, StringBuilder buf)

+			throws Exception {

+		BufferedReader result = new BufferedReader(new FileReader(file));

+		while (true) {

+			String s = result.readLine();

+			if (s == null) {

+				break;

+			} else {

+				buf.append(s);

+				buf.append('\n');

+			}

+		}

+		result.close();

+	}

 

-    protected static void writeStringToFile(File file, StringWriter buf) throws Exception {

-        PrintWriter result = new PrintWriter(new FileWriter(file));

-        result.print(buf);

-        result.close();

-    }

+	protected static void writeStringToFile(File file, StringWriter buf)

+			throws Exception {

+		PrintWriter result = new PrintWriter(new FileWriter(file));

+		result.print(buf);

+		result.close();

+	}

 

-    protected static void writeStringToFile(File file, StringBuilder buf) throws Exception {

-        PrintWriter result = new PrintWriter(new FileWriter(file));

-        result.print(buf);

-        result.close();

-    }

+	protected static void writeStringToFile(File file, StringBuilder buf)

+			throws Exception {

+		PrintWriter result = new PrintWriter(new FileWriter(file));

+		result.print(buf);

+		result.close();

+	}

 

-    protected static String removeExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot);

-    }

+	protected static String removeExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot);

+	}

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
index df947ea..05474ca 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
@@ -29,179 +29,182 @@
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;

 import edu.uci.ics.hyracks.control.nc.NodeControllerService;

 

+@SuppressWarnings("deprecation")

 public abstract class AbstractPerfTestSuiteClass extends TestSuite {

 

-    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/perf/hadoop/conf";

-    private static final String PATH_TO_HIVE_CONF = "src/test/resources/perf/hive/conf/hive-default.xml";

+	private static final String PATH_TO_HADOOP_CONF = "src/test/resources/perf/hadoop/conf";

+	private static final String PATH_TO_HIVE_CONF = "src/test/resources/perf/hive/conf/hive-default.xml";

+	private static final String PATH_TO_DATA = "src/test/resources/perf/data/";

 

-    private static final String PATH_TO_CLUSTER_CONF = "src/test/resources/perf/conf/cluster";

-    private static final String PATH_TO_DATA = "src/test/resources/perf/data/";

+	private MiniDFSCluster dfsCluster;

+	private MiniMRCluster mrCluster;

 

-    private MiniDFSCluster dfsCluster;

-    private MiniMRCluster mrCluster;

+	private JobConf conf = new JobConf();

+	protected FileSystem dfs;

 

-    private JobConf conf = new JobConf();

-    protected FileSystem dfs;

+	private int numberOfNC = 2;

+	private ClusterControllerService cc;

+	private Map<String, NodeControllerService> ncs = new HashMap<String, NodeControllerService>();

 

-    private int numberOfNC = 2;

-    private ClusterControllerService cc;

-    private Map<String, NodeControllerService> ncs = new HashMap<String, NodeControllerService>();

+	/**

+	 * setup cluster

+	 * 

+	 * @throws IOException

+	 */

+	protected void setup() throws Exception {

+		setupHdfs();

+		setupHyracks();

+	}

 

-    /**

-     * setup cluster

-     * 

-     * @throws IOException

-     */

-    protected void setup() throws Exception {

-        setupHdfs();

-        setupHyracks();

-    }

+	private void setupHdfs() throws IOException {

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));

+		HiveConf hconf = new HiveConf(SessionState.class);

+		hconf.addResource(new Path(PATH_TO_HIVE_CONF));

 

-    private void setupHdfs() throws IOException {

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));

-        HiveConf hconf = new HiveConf(SessionState.class);

-        hconf.addResource(new Path(PATH_TO_HIVE_CONF));

+		FileSystem lfs = FileSystem.getLocal(new Configuration());

+		lfs.delete(new Path("build"), true);

+		lfs.delete(new Path("metastore_db"), true);

 

-        FileSystem lfs = FileSystem.getLocal(new Configuration());

-        lfs.delete(new Path("build"), true);

-        lfs.delete(new Path("metastore_db"), true);

+		System.setProperty("hadoop.log.dir", "logs");

+		dfsCluster = new MiniDFSCluster(hconf, numberOfNC, true, null);

+		dfs = dfsCluster.getFileSystem();

 

-        System.setProperty("hadoop.log.dir", "logs");

-        dfsCluster = new MiniDFSCluster(hconf, numberOfNC, true, null);

-        dfs = dfsCluster.getFileSystem();

+		mrCluster = new MiniMRCluster(2, dfs.getUri().toString(), 1);

+		hconf.setVar(HiveConf.ConfVars.HADOOPJT,

+				"localhost:" + mrCluster.getJobTrackerPort());

+		hconf.setInt("mapred.min.split.size", 1342177280);

 

-        mrCluster = new MiniMRCluster(2, dfs.getUri().toString(), 1);

-        hconf.setVar(HiveConf.ConfVars.HADOOPJT, "localhost:" + mrCluster.getJobTrackerPort());

-        hconf.setInt("mapred.min.split.size", 1342177280);

+		conf = new JobConf(hconf);

+		ConfUtil.setJobConf(conf);

 

-        conf = new JobConf(hconf);

-        ConfUtil.setJobConf(conf);

+		String fsName = conf.get("fs.default.name");

+		hconf.set("hive.metastore.warehouse.dir",

+				fsName.concat("/tmp/hivesterix"));

+		String warehouse = hconf.get("hive.metastore.warehouse.dir");

+		dfs.mkdirs(new Path(warehouse));

+		ConfUtil.setHiveConf(hconf);

+	}

 

-        String fsName = conf.get("fs.default.name");

-        hconf.set("hive.metastore.warehouse.dir", fsName.concat("/tmp/hivesterix"));

-        String warehouse = hconf.get("hive.metastore.warehouse.dir");

-        dfs.mkdirs(new Path(warehouse));

-        ConfUtil.setHiveConf(hconf);

-    }

+	private void setupHyracks() throws Exception {

+		// read hive conf

+		HiveConf hconf = new HiveConf(SessionState.class);

+		hconf.addResource(new Path(PATH_TO_HIVE_CONF));

+		SessionState.start(hconf);

+		String ipAddress = hconf.get("hive.hyracks.host");

+		int clientPort = Integer.parseInt(hconf.get("hive.hyracks.port"));

+		int clusterPort = clientPort;

+		String applicationName = hconf.get("hive.hyracks.app");

 

-    private void setupHyracks() throws Exception {

-        // read hive conf

-        HiveConf hconf = new HiveConf(SessionState.class);

-        hconf.addResource(new Path(PATH_TO_HIVE_CONF));

-        SessionState.start(hconf);

-        String ipAddress = hconf.get("hive.hyracks.host");

-        int clientPort = Integer.parseInt(hconf.get("hive.hyracks.port"));

-        int clusterPort = clientPort;

-        String applicationName = hconf.get("hive.hyracks.app");

+		// start hyracks cc

+		CCConfig ccConfig = new CCConfig();

+		ccConfig.clientNetIpAddress = ipAddress;

+		ccConfig.clientNetPort = clientPort;

+		ccConfig.clusterNetPort = clusterPort;

+		ccConfig.profileDumpPeriod = 1000;

+		ccConfig.heartbeatPeriod = 200000000;

+		ccConfig.maxHeartbeatLapsePeriods = 200000000;

+		cc = new ClusterControllerService(ccConfig);

+		cc.start();

 

-        // start hyracks cc

-        CCConfig ccConfig = new CCConfig();

-        ccConfig.clientNetIpAddress = ipAddress;

-        ccConfig.clientNetPort = clientPort;

-        ccConfig.clusterNetPort = clusterPort;

-        ccConfig.profileDumpPeriod = 1000;

-        ccConfig.heartbeatPeriod = 200000000;

-        ccConfig.maxHeartbeatLapsePeriods = 200000000;

-        cc = new ClusterControllerService(ccConfig);

-        cc.start();

+		// start hyracks nc

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

+			NCConfig ncConfig = new NCConfig();

+			ncConfig.ccHost = ipAddress;

+			ncConfig.clusterNetIPAddress = ipAddress;

+			ncConfig.ccPort = clientPort;

+			ncConfig.dataIPAddress = "127.0.0.1";

+			ncConfig.nodeId = "nc" + i;

+			NodeControllerService nc = new NodeControllerService(ncConfig);

+			nc.start();

+			ncs.put(ncConfig.nodeId, nc);

+		}

 

-        // start hyracks nc

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

-            NCConfig ncConfig = new NCConfig();

-            ncConfig.ccHost = ipAddress;

-            ncConfig.clusterNetIPAddress = ipAddress;

-            ncConfig.ccPort = clientPort;

-            ncConfig.dataIPAddress = "127.0.0.1";

-            ncConfig.nodeId = "nc" + i;

-            NodeControllerService nc = new NodeControllerService(ncConfig);

-            nc.start();

-            ncs.put(ncConfig.nodeId, nc);

-        }

+		IHyracksClientConnection hcc = new HyracksConnection(

+				ccConfig.clientNetIpAddress, clientPort);

+		hcc.createApplication(applicationName, null);

+	}

 

-        IHyracksClientConnection hcc = new HyracksConnection(ccConfig.clientNetIpAddress, clientPort);

-        hcc.createApplication(applicationName, null);

-    }

+	protected void makeDir(String path) throws IOException {

+		dfs.mkdirs(new Path(path));

+	}

 

-    protected void makeDir(String path) throws IOException {

-        dfs.mkdirs(new Path(path));

-    }

+	protected void loadFiles(String src, String dest) throws IOException {

+		dfs.copyFromLocalFile(new Path(src), new Path(dest));

+	}

 

-    protected void loadFiles(String src, String dest) throws IOException {

-        dfs.copyFromLocalFile(new Path(src), new Path(dest));

-    }

+	protected void cleanup() throws Exception {

+		cleanupHdfs();

+		cleanupHyracks();

+	}

 

-    protected void cleanup() throws Exception {

-        cleanupHdfs();

-        cleanupHyracks();

-    }

+	/**

+	 * cleanup hdfs cluster

+	 */

+	private void cleanupHdfs() throws IOException {

+		dfs.delete(new Path("/"), true);

+		FileSystem.closeAll();

+		dfsCluster.shutdown();

+	}

 

-    /**

-     * cleanup hdfs cluster

-     */

-    private void cleanupHdfs() throws IOException {

-        dfs.delete(new Path("/"), true);

-        FileSystem.closeAll();

-        dfsCluster.shutdown();

-    }

+	/**

+	 * cleanup hyracks cluster

+	 */

+	private void cleanupHyracks() throws Exception {

+		Iterator<NodeControllerService> iterator = ncs.values().iterator();

+		while (iterator.hasNext()) {

+			NodeControllerService nc = iterator.next();

+			nc.stop();

+		}

+		cc.stop();

+	}

 

-    /**

-     * cleanup hyracks cluster

-     */

-    private void cleanupHyracks() throws Exception {

-        Iterator<NodeControllerService> iterator = ncs.values().iterator();

-        while (iterator.hasNext()) {

-            NodeControllerService nc = iterator.next();

-            nc.stop();

-        }

-        cc.stop();

-    }

+	protected static List<String> getIgnoreList(String ignorePath)

+			throws FileNotFoundException, IOException {

+		BufferedReader reader = new BufferedReader(new FileReader(ignorePath));

+		String s = null;

+		List<String> ignores = new ArrayList<String>();

+		while ((s = reader.readLine()) != null) {

+			ignores.add(s);

+		}

+		reader.close();

+		return ignores;

+	}

 

-    protected static List<String> getIgnoreList(String ignorePath) throws FileNotFoundException, IOException {

-        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));

-        String s = null;

-        List<String> ignores = new ArrayList<String>();

-        while ((s = reader.readLine()) != null) {

-            ignores.add(s);

-        }

-        reader.close();

-        return ignores;

-    }

+	protected static boolean isIgnored(String q, List<String> ignoreList) {

+		for (String ignore : ignoreList) {

+			if (ignore.equals(q)) {

+				return true;

+			}

+		}

+		return false;

+	}

 

-    protected static boolean isIgnored(String q, List<String> ignoreList) {

-        for (String ignore : ignoreList) {

-            if (ignore.equals(q)) {

-                return true;

-            }

-        }

-        return false;

-    }

+	protected void loadData() throws IOException {

 

-    protected void loadData() throws IOException {

+		makeDir("/tpch");

+		makeDir("/tpch/customer");

+		makeDir("/tpch/lineitem");

+		makeDir("/tpch/orders");

+		makeDir("/tpch/part");

+		makeDir("/tpch/partsupp");

+		makeDir("/tpch/supplier");

+		makeDir("/tpch/nation");

+		makeDir("/tpch/region");

 

-        makeDir("/tpch");

-        makeDir("/tpch/customer");

-        makeDir("/tpch/lineitem");

-        makeDir("/tpch/orders");

-        makeDir("/tpch/part");

-        makeDir("/tpch/partsupp");

-        makeDir("/tpch/supplier");

-        makeDir("/tpch/nation");

-        makeDir("/tpch/region");

+		makeDir("/jarod");

 

-        makeDir("/jarod");

+		loadFiles(PATH_TO_DATA + "customer.tbl", "/tpch/customer/");

+		loadFiles(PATH_TO_DATA + "lineitem.tbl", "/tpch/lineitem/");

+		loadFiles(PATH_TO_DATA + "orders.tbl", "/tpch/orders/");

+		loadFiles(PATH_TO_DATA + "part.tbl", "/tpch/part/");

+		loadFiles(PATH_TO_DATA + "partsupp.tbl", "/tpch/partsupp/");

+		loadFiles(PATH_TO_DATA + "supplier.tbl", "/tpch/supplier/");

+		loadFiles(PATH_TO_DATA + "nation.tbl", "/tpch/nation/");

+		loadFiles(PATH_TO_DATA + "region.tbl", "/tpch/region/");

 

-        loadFiles(PATH_TO_DATA + "customer.tbl", "/tpch/customer/");

-        loadFiles(PATH_TO_DATA + "lineitem.tbl", "/tpch/lineitem/");

-        loadFiles(PATH_TO_DATA + "orders.tbl", "/tpch/orders/");

-        loadFiles(PATH_TO_DATA + "part.tbl", "/tpch/part/");

-        loadFiles(PATH_TO_DATA + "partsupp.tbl", "/tpch/partsupp/");

-        loadFiles(PATH_TO_DATA + "supplier.tbl", "/tpch/supplier/");

-        loadFiles(PATH_TO_DATA + "nation.tbl", "/tpch/nation/");

-        loadFiles(PATH_TO_DATA + "region.tbl", "/tpch/region/");

-

-        loadFiles(PATH_TO_DATA + "ext-gby.tbl", "/jarod/");

-    }

+		loadFiles(PATH_TO_DATA + "ext-gby.tbl", "/jarod/");

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractHivesterixTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractHivesterixTestCase.java
index ae5fa05..560cef7 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractHivesterixTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractHivesterixTestCase.java
@@ -10,40 +10,43 @@
 import junit.framework.TestCase;

 

 public class AbstractHivesterixTestCase extends TestCase {

-    protected File queryFile;

+	protected File queryFile;

 

-    public AbstractHivesterixTestCase(String testName, File queryFile) {

-        super(testName);

-    }

+	public AbstractHivesterixTestCase(String testName, File queryFile) {

+		super(testName);

+	}

 

-    protected static void readFileToString(File file, StringBuilder buf) throws Exception {

-        BufferedReader result = new BufferedReader(new FileReader(file));

-        while (true) {

-            String s = result.readLine();

-            if (s == null) {

-                break;

-            } else {

-                buf.append(s);

-                buf.append('\n');

-            }

-        }

-        result.close();

-    }

+	protected static void readFileToString(File file, StringBuilder buf)

+			throws Exception {

+		BufferedReader result = new BufferedReader(new FileReader(file));

+		while (true) {

+			String s = result.readLine();

+			if (s == null) {

+				break;

+			} else {

+				buf.append(s);

+				buf.append('\n');

+			}

+		}

+		result.close();

+	}

 

-    protected static void writeStringToFile(File file, StringWriter buf) throws Exception {

-        PrintWriter result = new PrintWriter(new FileWriter(file));

-        result.print(buf);

-        result.close();

-    }

+	protected static void writeStringToFile(File file, StringWriter buf)

+			throws Exception {

+		PrintWriter result = new PrintWriter(new FileWriter(file));

+		result.print(buf);

+		result.close();

+	}

 

-    protected static void writeStringToFile(File file, StringBuilder buf) throws Exception {

-        PrintWriter result = new PrintWriter(new FileWriter(file));

-        result.print(buf);

-        result.close();

-    }

+	protected static void writeStringToFile(File file, StringBuilder buf)

+			throws Exception {

+		PrintWriter result = new PrintWriter(new FileWriter(file));

+		result.print(buf);

+		result.close();

+	}

 

-    protected static String removeExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot);

-    }

+	protected static String removeExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot);

+	}

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
index 74bde3f..e9a5736 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
@@ -30,182 +30,188 @@
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;

 import edu.uci.ics.hyracks.control.nc.NodeControllerService;

 

+@SuppressWarnings("deprecation")

 public abstract class AbstractTestSuiteClass extends TestSuite {

 

-    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/runtimefunctionts/hadoop/conf";

-    private static final String PATH_TO_HIVE_CONF = "src/test/resources/runtimefunctionts/hive/conf/hive-default.xml";

+	private static final String PATH_TO_HADOOP_CONF = "src/test/resources/runtimefunctionts/hadoop/conf";

+	private static final String PATH_TO_HIVE_CONF = "src/test/resources/runtimefunctionts/hive/conf/hive-default.xml";

 

-    private static final String PATH_TO_CLUSTER_CONF = "src/test/resources/runtimefunctionts/hive/conf/topology.xml";

-    private static final String PATH_TO_DATA = "src/test/resources/runtimefunctionts/data/";

+	private static final String PATH_TO_CLUSTER_CONF = "src/test/resources/runtimefunctionts/hive/conf/topology.xml";

+	private static final String PATH_TO_DATA = "src/test/resources/runtimefunctionts/data/";

 

-    private MiniDFSCluster dfsCluster;

-    private MiniMRCluster mrCluster;

+	private MiniDFSCluster dfsCluster;

+	private MiniMRCluster mrCluster;

 

-    private JobConf conf = new JobConf();

-    protected FileSystem dfs;

+	private JobConf conf = new JobConf();

+	protected FileSystem dfs;

 

-    private int numberOfNC = 2;

-    private ClusterControllerService cc;

-    private Map<String, NodeControllerService> ncs = new HashMap<String, NodeControllerService>();

+	private int numberOfNC = 2;

+	private ClusterControllerService cc;

+	private Map<String, NodeControllerService> ncs = new HashMap<String, NodeControllerService>();

 

-    /**

-     * setup cluster

-     * 

-     * @throws IOException

-     */

-    protected void setup() throws Exception {

-        setupHdfs();

-        setupHyracks();

-    }

+	/**

+	 * setup cluster

+	 * 

+	 * @throws IOException

+	 */

+	protected void setup() throws Exception {

+		setupHdfs();

+		setupHyracks();

+	}

 

-    private void setupHdfs() throws IOException {

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));

-        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));

-        HiveConf hconf = new HiveConf(SessionState.class);

-        hconf.addResource(new Path(PATH_TO_HIVE_CONF));

+	private void setupHdfs() throws IOException {

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));

+		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));

+		HiveConf hconf = new HiveConf(SessionState.class);

+		hconf.addResource(new Path(PATH_TO_HIVE_CONF));

 

-        FileSystem lfs = FileSystem.getLocal(new Configuration());

-        lfs.delete(new Path("build"), true);

-        lfs.delete(new Path("metastore_db"), true);

+		FileSystem lfs = FileSystem.getLocal(new Configuration());

+		lfs.delete(new Path("build"), true);

+		lfs.delete(new Path("metastore_db"), true);

 

-        System.setProperty("hadoop.log.dir", "logs");

-        dfsCluster = new MiniDFSCluster(hconf, numberOfNC, true, null);

-        dfs = dfsCluster.getFileSystem();

+		System.setProperty("hadoop.log.dir", "logs");

+		dfsCluster = new MiniDFSCluster(hconf, numberOfNC, true, null);

+		dfs = dfsCluster.getFileSystem();

 

-        mrCluster = new MiniMRCluster(2, dfs.getUri().toString(), 1);

-        hconf.setVar(HiveConf.ConfVars.HADOOPJT, "localhost:" + mrCluster.getJobTrackerPort());

+		mrCluster = new MiniMRCluster(2, dfs.getUri().toString(), 1);

+		hconf.setVar(HiveConf.ConfVars.HADOOPJT,

+				"localhost:" + mrCluster.getJobTrackerPort());

 

-        conf = new JobConf(hconf);

-        ConfUtil.setJobConf(conf);

+		conf = new JobConf(hconf);

+		ConfUtil.setJobConf(conf);

 

-        String fsName = conf.get("fs.default.name");

-        hconf.set("hive.metastore.warehouse.dir", fsName.concat("/tmp/hivesterix"));

-        String warehouse = hconf.get("hive.metastore.warehouse.dir");

-        dfs.mkdirs(new Path(warehouse));

-        ConfUtil.setHiveConf(hconf);

-    }

+		String fsName = conf.get("fs.default.name");

+		hconf.set("hive.metastore.warehouse.dir",

+				fsName.concat("/tmp/hivesterix"));

+		String warehouse = hconf.get("hive.metastore.warehouse.dir");

+		dfs.mkdirs(new Path(warehouse));

+		ConfUtil.setHiveConf(hconf);

+	}

 

-    private void setupHyracks() throws Exception {

-        // read hive conf

-        HiveConf hconf = new HiveConf(SessionState.class);

-        hconf.addResource(new Path(PATH_TO_HIVE_CONF));

-        SessionState.start(hconf);

-        String ipAddress = hconf.get("hive.hyracks.host");

-        int clientPort = Integer.parseInt(hconf.get("hive.hyracks.port"));

-        int netPort = clientPort + 1;

-        String applicationName = hconf.get("hive.hyracks.app");

+	private void setupHyracks() throws Exception {

+		// read hive conf

+		HiveConf hconf = new HiveConf(SessionState.class);

+		hconf.addResource(new Path(PATH_TO_HIVE_CONF));

+		SessionState.start(hconf);

+		String ipAddress = hconf.get("hive.hyracks.host");

+		int clientPort = Integer.parseInt(hconf.get("hive.hyracks.port"));

+		int netPort = clientPort + 1;

+		String applicationName = hconf.get("hive.hyracks.app");

 

-        // start hyracks cc

-        CCConfig ccConfig = new CCConfig();

-        ccConfig.clientNetIpAddress = ipAddress;

-        ccConfig.clientNetPort = clientPort;

-        ccConfig.clusterNetPort = netPort;

-        ccConfig.profileDumpPeriod = 1000;

-        ccConfig.heartbeatPeriod = 200000000;

-        ccConfig.maxHeartbeatLapsePeriods = 200000000;

-        ccConfig.clusterTopologyDefinition = new File(PATH_TO_CLUSTER_CONF);

-        cc = new ClusterControllerService(ccConfig);

-        cc.start();

+		// start hyracks cc

+		CCConfig ccConfig = new CCConfig();

+		ccConfig.clientNetIpAddress = ipAddress;

+		ccConfig.clientNetPort = clientPort;

+		ccConfig.clusterNetPort = netPort;

+		ccConfig.profileDumpPeriod = 1000;

+		ccConfig.heartbeatPeriod = 200000000;

+		ccConfig.maxHeartbeatLapsePeriods = 200000000;

+		ccConfig.clusterTopologyDefinition = new File(PATH_TO_CLUSTER_CONF);

+		cc = new ClusterControllerService(ccConfig);

+		cc.start();

 

-        // start hyracks nc

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

-            NCConfig ncConfig = new NCConfig();

-            ncConfig.ccHost = ipAddress;

-            ncConfig.clusterNetIPAddress = ipAddress;

-            ncConfig.ccPort = netPort;

-            ncConfig.dataIPAddress = "127.0.0.1";

-            ncConfig.nodeId = "nc" + i;

-            NodeControllerService nc = new NodeControllerService(ncConfig);

-            nc.start();

-            ncs.put(ncConfig.nodeId, nc);

-        }

+		// start hyracks nc

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

+			NCConfig ncConfig = new NCConfig();

+			ncConfig.ccHost = ipAddress;

+			ncConfig.clusterNetIPAddress = ipAddress;

+			ncConfig.ccPort = netPort;

+			ncConfig.dataIPAddress = "127.0.0.1";

+			ncConfig.nodeId = "nc" + i;

+			NodeControllerService nc = new NodeControllerService(ncConfig);

+			nc.start();

+			ncs.put(ncConfig.nodeId, nc);

+		}

 

-        IHyracksClientConnection hcc = new HyracksConnection(ccConfig.clientNetIpAddress, clientPort);

-        hcc.createApplication(applicationName, null);

-    }

+		IHyracksClientConnection hcc = new HyracksConnection(

+				ccConfig.clientNetIpAddress, clientPort);

+		hcc.createApplication(applicationName, null);

+	}

 

-    protected void makeDir(String path) throws IOException {

-        dfs.mkdirs(new Path(path));

-    }

+	protected void makeDir(String path) throws IOException {

+		dfs.mkdirs(new Path(path));

+	}

 

-    protected void loadFiles(String src, String dest) throws IOException {

-        dfs.copyFromLocalFile(new Path(src), new Path(dest));

-    }

+	protected void loadFiles(String src, String dest) throws IOException {

+		dfs.copyFromLocalFile(new Path(src), new Path(dest));

+	}

 

-    protected void cleanup() throws Exception {

-        cleanupHdfs();

-        cleanupHyracks();

-    }

+	protected void cleanup() throws Exception {

+		cleanupHdfs();

+		cleanupHyracks();

+	}

 

-    /**

-     * cleanup hdfs cluster

-     */

-    private void cleanupHdfs() throws IOException {

-        dfs.delete(new Path("/"), true);

-        FileSystem.closeAll();

-        dfsCluster.shutdown();

-    }

+	/**

+	 * cleanup hdfs cluster

+	 */

+	private void cleanupHdfs() throws IOException {

+		dfs.delete(new Path("/"), true);

+		FileSystem.closeAll();

+		dfsCluster.shutdown();

+	}

 

-    /**

-     * cleanup hyracks cluster

-     */

-    private void cleanupHyracks() throws Exception {

-        Iterator<NodeControllerService> iterator = ncs.values().iterator();

-        while (iterator.hasNext()) {

-            NodeControllerService nc = iterator.next();

-            nc.stop();

-        }

-        cc.stop();

-    }

+	/**

+	 * cleanup hyracks cluster

+	 */

+	private void cleanupHyracks() throws Exception {

+		Iterator<NodeControllerService> iterator = ncs.values().iterator();

+		while (iterator.hasNext()) {

+			NodeControllerService nc = iterator.next();

+			nc.stop();

+		}

+		cc.stop();

+	}

 

-    protected static List<String> getIgnoreList(String ignorePath) throws FileNotFoundException, IOException {

-        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));

-        String s = null;

-        List<String> ignores = new ArrayList<String>();

-        while ((s = reader.readLine()) != null) {

-            ignores.add(s);

-        }

-        reader.close();

-        return ignores;

-    }

+	protected static List<String> getIgnoreList(String ignorePath)

+			throws FileNotFoundException, IOException {

+		BufferedReader reader = new BufferedReader(new FileReader(ignorePath));

+		String s = null;

+		List<String> ignores = new ArrayList<String>();

+		while ((s = reader.readLine()) != null) {

+			ignores.add(s);

+		}

+		reader.close();

+		return ignores;

+	}

 

-    protected static boolean isIgnored(String q, List<String> ignoreList) {

-        for (String ignore : ignoreList) {

-            if (ignore.equals(q)) {

-                return true;

-            }

-        }

-        return false;

-    }

+	protected static boolean isIgnored(String q, List<String> ignoreList) {

+		for (String ignore : ignoreList) {

+			if (q.indexOf(ignore) >= 0) {

+				return true;

+			}

+		}

+		return false;

+	}

 

-    protected void loadData() throws IOException {

+	protected void loadData() throws IOException {

 

-        makeDir("/tpch");

-        makeDir("/tpch/customer");

-        makeDir("/tpch/lineitem");

-        makeDir("/tpch/orders");

-        makeDir("/tpch/part");

-        makeDir("/tpch/partsupp");

-        makeDir("/tpch/supplier");

-        makeDir("/tpch/nation");

-        makeDir("/tpch/region");

+		makeDir("/tpch");

+		makeDir("/tpch/customer");

+		makeDir("/tpch/lineitem");

+		makeDir("/tpch/orders");

+		makeDir("/tpch/part");

+		makeDir("/tpch/partsupp");

+		makeDir("/tpch/supplier");

+		makeDir("/tpch/nation");

+		makeDir("/tpch/region");

 

-        makeDir("/test");

-        makeDir("/test/joinsrc1");

-        makeDir("/test/joinsrc2");

+		makeDir("/test");

+		makeDir("/test/joinsrc1");

+		makeDir("/test/joinsrc2");

 

-        loadFiles(PATH_TO_DATA + "customer.tbl", "/tpch/customer/");

-        loadFiles(PATH_TO_DATA + "lineitem.tbl", "/tpch/lineitem/");

-        loadFiles(PATH_TO_DATA + "orders.tbl", "/tpch/orders/");

-        loadFiles(PATH_TO_DATA + "part.tbl", "/tpch/part/");

-        loadFiles(PATH_TO_DATA + "partsupp.tbl", "/tpch/partsupp/");

-        loadFiles(PATH_TO_DATA + "supplier.tbl", "/tpch/supplier/");

-        loadFiles(PATH_TO_DATA + "nation.tbl", "/tpch/nation/");

-        loadFiles(PATH_TO_DATA + "region.tbl", "/tpch/region/");

+		loadFiles(PATH_TO_DATA + "customer.tbl", "/tpch/customer/");

+		loadFiles(PATH_TO_DATA + "lineitem.tbl", "/tpch/lineitem/");

+		loadFiles(PATH_TO_DATA + "orders.tbl", "/tpch/orders/");

+		loadFiles(PATH_TO_DATA + "part.tbl", "/tpch/part/");

+		loadFiles(PATH_TO_DATA + "partsupp.tbl", "/tpch/partsupp/");

+		loadFiles(PATH_TO_DATA + "supplier.tbl", "/tpch/supplier/");

+		loadFiles(PATH_TO_DATA + "nation.tbl", "/tpch/nation/");

+		loadFiles(PATH_TO_DATA + "region.tbl", "/tpch/region/");

 

-        loadFiles(PATH_TO_DATA + "large_card_join_src.tbl", "/test/joinsrc1/");

-        loadFiles(PATH_TO_DATA + "large_card_join_src_small.tbl", "/test/joinsrc2/");

-    }

+		loadFiles(PATH_TO_DATA + "large_card_join_src.tbl", "/test/joinsrc1/");

+		loadFiles(PATH_TO_DATA + "large_card_join_src_small.tbl",

+				"/test/joinsrc2/");

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/datagen/RecordBalance.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/datagen/RecordBalance.java
index e5746bb..800d6be 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/datagen/RecordBalance.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/datagen/RecordBalance.java
@@ -18,57 +18,65 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
 
+@SuppressWarnings("deprecation")
 public class RecordBalance {
 
-    private static String confPath = System.getenv("HADDOP_HOME");
-    private static Path[] inputPaths = { new Path("/tpch/100x/customer"), new Path("/tpch/100x/nation"),
-            new Path("/tpch/100x/region"), new Path("/tpch/100x/lineitem"), new Path("/tpch/100x/orders"),
-            new Path("/tpch/100x/part"), new Path("/tpch/100x/partsupp"), new Path("/tpch/100x/supplier") };
+	private static String confPath = System.getenv("HADDOP_HOME");
+	private static Path[] inputPaths = { new Path("/tpch/100x/customer"),
+			new Path("/tpch/100x/nation"), new Path("/tpch/100x/region"),
+			new Path("/tpch/100x/lineitem"), new Path("/tpch/100x/orders"),
+			new Path("/tpch/100x/part"), new Path("/tpch/100x/partsupp"),
+			new Path("/tpch/100x/supplier") };
 
-    private static Path[] outputPaths = { new Path("/tpch/100/customer"), new Path("/tpch/100/nation"),
-            new Path("/tpch/100/region"), new Path("/tpch/100/lineitem"), new Path("/tpch/100/orders"),
-            new Path("/tpch/100/part"), new Path("/tpch/100/partsupp"), new Path("/tpch/100/supplier") };
+	private static Path[] outputPaths = { new Path("/tpch/100/customer"),
+			new Path("/tpch/100/nation"), new Path("/tpch/100/region"),
+			new Path("/tpch/100/lineitem"), new Path("/tpch/100/orders"),
+			new Path("/tpch/100/part"), new Path("/tpch/100/partsupp"),
+			new Path("/tpch/100/supplier") };
 
-    public static class MapRecordOnly extends MapReduceBase implements Mapper<LongWritable, Text, LongWritable, Text> {
+	public static class MapRecordOnly extends MapReduceBase implements
+			Mapper<LongWritable, Text, LongWritable, Text> {
 
-        public void map(LongWritable id, Text inputValue, OutputCollector<LongWritable, Text> output, Reporter reporter)
-                throws IOException {
-            output.collect(id, inputValue);
-        }
-    }
+		public void map(LongWritable id, Text inputValue,
+				OutputCollector<LongWritable, Text> output, Reporter reporter)
+				throws IOException {
+			output.collect(id, inputValue);
+		}
+	}
 
-    public static class ReduceRecordOnly extends MapReduceBase implements
-            Reducer<LongWritable, Text, NullWritable, Text> {
+	public static class ReduceRecordOnly extends MapReduceBase implements
+			Reducer<LongWritable, Text, NullWritable, Text> {
 
-        NullWritable key = NullWritable.get();
+		NullWritable key = NullWritable.get();
 
-        public void reduce(LongWritable inputKey, Iterator<Text> inputValue,
-                OutputCollector<NullWritable, Text> output, Reporter reporter) throws IOException {
-            while (inputValue.hasNext())
-                output.collect(key, inputValue.next());
-        }
-    }
+		public void reduce(LongWritable inputKey, Iterator<Text> inputValue,
+				OutputCollector<NullWritable, Text> output, Reporter reporter)
+				throws IOException {
+			while (inputValue.hasNext())
+				output.collect(key, inputValue.next());
+		}
+	}
 
-    public static void main(String[] args) throws IOException {
+	public static void main(String[] args) throws IOException {
 
-        for (int i = 0; i < inputPaths.length; i++) {
-            JobConf job = new JobConf(RecordBalance.class);
-            job.addResource(new Path(confPath + "/core-site.xml"));
-            job.addResource(new Path(confPath + "/mapred-site.xml"));
-            job.addResource(new Path(confPath + "/hdfs-site.xml"));
+		for (int i = 0; i < inputPaths.length; i++) {
+			JobConf job = new JobConf(RecordBalance.class);
+			job.addResource(new Path(confPath + "/core-site.xml"));
+			job.addResource(new Path(confPath + "/mapred-site.xml"));
+			job.addResource(new Path(confPath + "/hdfs-site.xml"));
 
-            job.setJobName(RecordBalance.class.getSimpleName());
-            job.setMapperClass(MapRecordOnly.class);
-            job.setReducerClass(ReduceRecordOnly.class);
-            job.setMapOutputKeyClass(LongWritable.class);
-            job.setMapOutputValueClass(Text.class);
+			job.setJobName(RecordBalance.class.getSimpleName());
+			job.setMapperClass(MapRecordOnly.class);
+			job.setReducerClass(ReduceRecordOnly.class);
+			job.setMapOutputKeyClass(LongWritable.class);
+			job.setMapOutputValueClass(Text.class);
 
-            job.setInputFormat(TextInputFormat.class);
-            FileInputFormat.setInputPaths(job, inputPaths[i]);
-            FileOutputFormat.setOutputPath(job, outputPaths[i]);
-            job.setNumReduceTasks(Integer.parseInt(args[0]));
+			job.setInputFormat(TextInputFormat.class);
+			FileInputFormat.setInputPaths(job, inputPaths[i]);
+			FileOutputFormat.setOutputPath(job, outputPaths[i]);
+			job.setNumReduceTasks(Integer.parseInt(args[0]));
 
-            JobClient.runJob(job);
-        }
-    }
+			JobClient.runJob(job);
+		}
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/legacy/LegacyTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/legacy/LegacyTestCase.java
index a286747..9591c32 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/legacy/LegacyTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/legacy/LegacyTestCase.java
@@ -15,128 +15,130 @@
 import edu.uci.ics.hivesterix.test.base.AbstractHivesterixTestCase;
 
 public class LegacyTestCase extends AbstractHivesterixTestCase {
-    private File resultFile;
-    private FileSystem dfs;
+	private File resultFile;
+	private FileSystem dfs;
 
-    public LegacyTestCase(File queryFile, File resultFile) {
-        super("legacy", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	public LegacyTestCase(File queryFile, File resultFile) {
+		super("legacy", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testRuntimeFunction() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	@Test
+	public void testRuntimeFunction() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
 
-        dfs = FileSystem.get(ConfUtil.getJobConf());
+		dfs = FileSystem.get(ConfUtil.getJobConf());
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            driver.run(query);
-            driver.clear();
-            i++;
-        }
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			driver.run(query);
+			driver.clear();
+			i++;
+		}
 
-        String warehouse = hconf.get("hive.metastore.warehouse.dir");
-        String tableName = removeExt(resultFile.getName());
-        String directory = warehouse + "/" + tableName + "/";
-        String localDirectory = "tmp";
+		String warehouse = hconf.get("hive.metastore.warehouse.dir");
+		String tableName = removeExt(resultFile.getName());
+		String directory = warehouse + "/" + tableName + "/";
+		String localDirectory = "tmp";
 
-        FileStatus[] files = dfs.listStatus(new Path(directory));
-        FileSystem lfs = null;
-        if (files == null) {
-            lfs = FileSystem.getLocal(ConfUtil.getJobConf());
-            files = lfs.listStatus(new Path(directory));
-        }
+		FileStatus[] files = dfs.listStatus(new Path(directory));
+		FileSystem lfs = null;
+		if (files == null) {
+			lfs = FileSystem.getLocal(ConfUtil.getJobConf());
+			files = lfs.listStatus(new Path(directory));
+		}
 
-        File resultDirectory = new File(localDirectory + "/" + tableName);
-        deleteDir(resultDirectory);
-        resultDirectory.mkdir();
+		File resultDirectory = new File(localDirectory + "/" + tableName);
+		deleteDir(resultDirectory);
+		resultDirectory.mkdir();
 
-        for (FileStatus fs : files) {
-            Path src = fs.getPath();
-            if (src.getName().indexOf("crc") >= 0)
-                continue;
+		for (FileStatus fs : files) {
+			Path src = fs.getPath();
+			if (src.getName().indexOf("crc") >= 0)
+				continue;
 
-            String destStr = localDirectory + "/" + tableName + "/" + src.getName();
-            Path dest = new Path(destStr);
-            if (lfs != null) {
-                lfs.copyToLocalFile(src, dest);
-                dfs.copyFromLocalFile(dest, new Path(directory));
-            } else
-                dfs.copyToLocalFile(src, dest);
-        }
+			String destStr = localDirectory + "/" + tableName + "/"
+					+ src.getName();
+			Path dest = new Path(destStr);
+			if (lfs != null) {
+				lfs.copyToLocalFile(src, dest);
+				dfs.copyFromLocalFile(dest, new Path(directory));
+			} else
+				dfs.copyToLocalFile(src, dest);
+		}
 
-        File[] rFiles = resultDirectory.listFiles();
-        StringBuilder sb = new StringBuilder();
-        for (File r : rFiles) {
-            if (r.getName().indexOf("crc") >= 0)
-                continue;
-            readFileToString(r, sb);
-        }
-        deleteDir(resultDirectory);
+		File[] rFiles = resultDirectory.listFiles();
+		StringBuilder sb = new StringBuilder();
+		for (File r : rFiles) {
+			if (r.getName().indexOf("crc") >= 0)
+				continue;
+			readFileToString(r, sb);
+		}
+		deleteDir(resultDirectory);
 
-        StringBuilder buf = new StringBuilder();
-        readFileToString(resultFile, buf);
-        if (!equal(buf, sb)) {
-            throw new Exception("Result for " + queryFile + " changed:\n" + sw.toString());
-        }
-    }
+		StringBuilder buf = new StringBuilder();
+		readFileToString(resultFile, buf);
+		if (!equal(buf, sb)) {
+			throw new Exception("Result for " + queryFile + " changed:\n"
+					+ sw.toString());
+		}
+	}
 
-    private void deleteDir(File resultDirectory) {
-        if (resultDirectory.exists()) {
-            File[] rFiles = resultDirectory.listFiles();
-            for (File r : rFiles)
-                r.delete();
-            resultDirectory.delete();
-        }
-    }
+	private void deleteDir(File resultDirectory) {
+		if (resultDirectory.exists()) {
+			File[] rFiles = resultDirectory.listFiles();
+			for (File r : rFiles)
+				r.delete();
+			resultDirectory.delete();
+		}
+	}
 
-    private boolean equal(StringBuilder sb1, StringBuilder sb2) {
-        String s1 = sb1.toString();
-        String s2 = sb2.toString();
-        String[] rowsOne = s1.split("\n");
-        String[] rowsTwo = s2.split("\n");
+	private boolean equal(StringBuilder sb1, StringBuilder sb2) {
+		String s1 = sb1.toString();
+		String s2 = sb2.toString();
+		String[] rowsOne = s1.split("\n");
+		String[] rowsTwo = s2.split("\n");
 
-        if (rowsOne.length != rowsTwo.length)
-            return false;
+		if (rowsOne.length != rowsTwo.length)
+			return false;
 
-        for (int i = 0; i < rowsOne.length; i++) {
-            String row1 = rowsOne[i];
-            String row2 = rowsTwo[i];
+		for (int i = 0; i < rowsOne.length; i++) {
+			String row1 = rowsOne[i];
+			String row2 = rowsTwo[i];
 
-            if (row1.equals(row2))
-                continue;
+			if (row1.equals(row2))
+				continue;
 
-            String[] fields1 = row1.split("");
-            String[] fields2 = row2.split("");
+			String[] fields1 = row1.split("");
+			String[] fields2 = row2.split("");
 
-            for (int j = 0; j < fields1.length; j++) {
-                if (fields1[j].equals(fields2[j])) {
-                    continue;
-                } else if (fields1[j].indexOf('.') < 0) {
-                    return false;
-                } else {
-                    Float float1 = Float.parseFloat(fields1[j]);
-                    Float float2 = Float.parseFloat(fields2[j]);
+			for (int j = 0; j < fields1.length; j++) {
+				if (fields1[j].equals(fields2[j])) {
+					continue;
+				} else if (fields1[j].indexOf('.') < 0) {
+					return false;
+				} else {
+					Float float1 = Float.parseFloat(fields1[j]);
+					Float float2 = Float.parseFloat(fields2[j]);
 
-                    if (Math.abs(float1 - float2) == 0)
-                        continue;
-                    else
-                        return false;
-                }
-            }
-        }
+					if (Math.abs(float1 - float2) == 0)
+						continue;
+					else
+						return false;
+				}
+			}
+		}
 
-        return true;
-    }
+		return true;
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
index f80405f..db13676 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
@@ -12,42 +12,46 @@
 import edu.uci.ics.hivesterix.test.base.AbstractHivesterixTestCase;
 
 public class OptimizerTestCase extends AbstractHivesterixTestCase {
-    private File resultFile;
+	private File resultFile;
 
-    OptimizerTestCase(File queryFile, File resultFile) {
-        super("testOptimizer", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	OptimizerTestCase(File queryFile, File resultFile) {
+		super("testOptimizer", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testOptimizer() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	@Test
+	public void testOptimizer() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            if (query.toLowerCase().indexOf("create") >= 0 || query.toLowerCase().indexOf("drop") >= 0
-                    || query.toLowerCase().indexOf("set") >= 0 || query.toLowerCase().startsWith("\n\ncreate")
-                    || query.toLowerCase().startsWith("\n\ndrop") || query.toLowerCase().startsWith("\n\nset"))
-                driver.run(query);
-            else
-                driver.compile(query);
-            driver.clear();
-            i++;
-        }
-        StringBuilder buf = new StringBuilder();
-        readFileToString(resultFile, buf);
-        if (!buf.toString().equals(sw.toString())) {
-            throw new Exception("Result for " + queryFile + " changed:\n" + sw.toString());
-        }
-    }
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			if (query.toLowerCase().indexOf("create") >= 0
+					|| query.toLowerCase().indexOf("drop") >= 0
+					|| query.toLowerCase().indexOf("set") >= 0
+					|| query.toLowerCase().startsWith("\n\ncreate")
+					|| query.toLowerCase().startsWith("\n\ndrop")
+					|| query.toLowerCase().startsWith("\n\nset"))
+				driver.run(query);
+			else
+				driver.compile(query);
+			driver.clear();
+			i++;
+		}
+		StringBuilder buf = new StringBuilder();
+		readFileToString(resultFile, buf);
+		if (!buf.toString().equals(sw.toString())) {
+			throw new Exception("Result for " + queryFile + " changed:\n"
+					+ sw.toString());
+		}
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuitGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuitGenerator.java
index c6b788f..217f67d 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuitGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuitGenerator.java
@@ -11,65 +11,67 @@
 import edu.uci.ics.hivesterix.test.base.AbstractTestSuiteClass;
 
 public class OptimizerTestSuitGenerator extends AbstractTestSuiteClass {
-    private static final String PATH_TO_QUERIES = "src/test/resources/optimizerts/queries/";
-    private static final String PATH_TO_RESULTS = "src/test/resources/optimizerts/results/";
-    private static final String PATH_TO_IGNORES = "src/test/resources/optimizerts/ignore.txt";
+	private static final String PATH_TO_QUERIES = "src/test/resources/optimizerts/queries/";
+	private static final String PATH_TO_RESULTS = "src/test/resources/optimizerts/results/";
+	private static final String PATH_TO_IGNORES = "src/test/resources/optimizerts/ignore.txt";
 
-    private static final String FILE_EXTENSION_OF_RESULTS = "plan";
+	private static final String FILE_EXTENSION_OF_RESULTS = "plan";
 
-    public static Test suite() throws UnsupportedEncodingException, FileNotFoundException, IOException {
-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
-        File testData = new File(PATH_TO_QUERIES);
-        File[] queries = testData.listFiles();
-        OptimizerTestSuitGenerator testSuite = new OptimizerTestSuitGenerator();
-        // set hdfs and hyracks cluster, and load test data to hdfs
-        try {
-            testSuite.setup();
-            testSuite.loadData();
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IllegalStateException(e.getMessage());
-        }
+	public static Test suite() throws UnsupportedEncodingException,
+			FileNotFoundException, IOException {
+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
+		File testData = new File(PATH_TO_QUERIES);
+		File[] queries = testData.listFiles();
+		OptimizerTestSuitGenerator testSuite = new OptimizerTestSuitGenerator();
+		// set hdfs and hyracks cluster, and load test data to hdfs
+		try {
+			testSuite.setup();
+			testSuite.loadData();
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new IllegalStateException(e.getMessage());
+		}
 
-        for (File qFile : queries) {
-            if (isIgnored(qFile.getName(), ignores))
-                continue;
+		for (File qFile : queries) {
+			if (isIgnored(qFile.getName(), ignores))
+				continue;
 
-            if (qFile.isFile()) {
-                String resultFileName = aqlExtToResExt(qFile.getName());
-                File rFile = new File(PATH_TO_RESULTS + resultFileName);
-                testSuite.addTest(new OptimizerTestSuiteCaseGenerator(qFile, rFile));
-            }
-        }
-        return testSuite;
-    }
+			if (qFile.isFile()) {
+				String resultFileName = aqlExtToResExt(qFile.getName());
+				File rFile = new File(PATH_TO_RESULTS + resultFileName);
+				testSuite.addTest(new OptimizerTestSuiteCaseGenerator(qFile,
+						rFile));
+			}
+		}
+		return testSuite;
+	}
 
-    private static String aqlExtToResExt(String fname) {
-        int dot = fname.lastIndexOf('.');
-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
-    }
+	private static String aqlExtToResExt(String fname) {
+		int dot = fname.lastIndexOf('.');
+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+	}
 
-    /**
-     * Runs the tests and collects their result in a TestResult.
-     */
-    @Override
-    public void run(TestResult result) {
+	/**
+	 * Runs the tests and collects their result in a TestResult.
+	 */
+	@Override
+	public void run(TestResult result) {
 
-        int testCount = countTestCases();
-        for (int i = 0; i < testCount; i++) {
-            Test each = this.testAt(i);
-            if (result.shouldStop())
-                break;
-            runTest(each, result);
-        }
+		int testCount = countTestCases();
+		for (int i = 0; i < testCount; i++) {
+			Test each = this.testAt(i);
+			if (result.shouldStop())
+				break;
+			runTest(each, result);
+		}
 
-        // cleanup hdfs and hyracks cluster
-        try {
-            cleanup();
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IllegalStateException(e.getMessage());
-        }
-    }
+		// cleanup hdfs and hyracks cluster
+		try {
+			cleanup();
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new IllegalStateException(e.getMessage());
+		}
+	}
 
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
index 8ac4e86..e3a4a4e 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
@@ -11,43 +11,44 @@
 
 public class OptimizerTestSuite extends AbstractTestSuiteClass {
 
-    private static final String PATH_TO_QUERIES = "src/test/resources/optimizerts/queries/";
-    private static final String PATH_TO_RESULTS = "src/test/resources/optimizerts/results/";
-    private static final String PATH_TO_IGNORES = "src/test/resources/optimizerts/ignore.txt";
+	private static final String PATH_TO_QUERIES = "src/test/resources/optimizerts/queries/";
+	private static final String PATH_TO_RESULTS = "src/test/resources/optimizerts/results/";
+	private static final String PATH_TO_IGNORES = "src/test/resources/optimizerts/ignore.txt";
 
-    private static final String FILE_EXTENSION_OF_RESULTS = "plan";
+	private static final String FILE_EXTENSION_OF_RESULTS = "plan";
 
-    public static Test suite() throws UnsupportedEncodingException, FileNotFoundException, IOException {
-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
-        File testData = new File(PATH_TO_QUERIES);
-        File[] queries = testData.listFiles();
-        OptimizerTestSuite testSuite = new OptimizerTestSuite();
+	public static Test suite() throws UnsupportedEncodingException,
+			FileNotFoundException, IOException {
+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);
+		File testData = new File(PATH_TO_QUERIES);
+		File[] queries = testData.listFiles();
+		OptimizerTestSuite testSuite = new OptimizerTestSuite();
 
-        // set hdfs and hyracks cluster, and load test data to hdfs
-        try {
-            testSuite.setup();
-            testSuite.loadData();
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IllegalStateException(e.getMessage());
-        }
+		// set hdfs and hyracks cluster, and load test data to hdfs
+		try {
+			testSuite.setup();
+			testSuite.loadData();
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new IllegalStateException(e.getMessage());
+		}
 
-        for (File qFile : queries) {
-            if (isIgnored(qFile.getName(), ignores))
-                continue;
+		for (File qFile : queries) {
+			if (isIgnored(qFile.getName(), ignores))
+				continue;
 
-            if (qFile.isFile() && qFile.getName().startsWith("h11_")) {
-                String resultFileName = hiveExtToResExt(qFile.getName());
-                File rFile = new File(PATH_TO_RESULTS + resultFileName);
-                testSuite.addTest(new OptimizerTestCase(qFile, rFile));
-            }
-        }
-        return testSuite;
-    }
+			if (qFile.isFile() && qFile.getName().startsWith("h11_")) {
+				String resultFileName = hiveExtToResExt(qFile.getName());
+				File rFile = new File(PATH_TO_RESULTS + resultFileName);
+				testSuite.addTest(new OptimizerTestCase(qFile, rFile));
+			}
+		}
+		return testSuite;
+	}
 
-    private static String hiveExtToResExt(String fname) {
-        int dot = fname.lastIndexOf('.');
-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
-    }
+	private static String hiveExtToResExt(String fname) {
+		int dot = fname.lastIndexOf('.');
+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+	}
 
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuiteCaseGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuiteCaseGenerator.java
index 6a3d242..a86dc29 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuiteCaseGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuiteCaseGenerator.java
@@ -12,40 +12,42 @@
 import edu.uci.ics.hivesterix.test.base.AbstractHivesterixTestCase;
 
 public class OptimizerTestSuiteCaseGenerator extends AbstractHivesterixTestCase {
-    private File resultFile;
-    private static final String PATH_TO_HIVE_CONF = "src/test/resources/runtimefunctionts/hive/conf/hive-default.xml";
+	private File resultFile;
 
-    OptimizerTestSuiteCaseGenerator(File queryFile, File resultFile) {
-        super("testOptimizer", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	OptimizerTestSuiteCaseGenerator(File queryFile, File resultFile) {
+		super("testOptimizer", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testOptimizer() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	@Test
+	public void testOptimizer() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            if (query.toLowerCase().indexOf("create") >= 0 || query.toLowerCase().indexOf("drop") >= 0
-                    || query.toLowerCase().indexOf("set") >= 0 || query.toLowerCase().startsWith("\n\ncreate")
-                    || query.toLowerCase().startsWith("\n\ndrop") || query.toLowerCase().startsWith("\n\nset"))
-                driver.run(query);
-            else
-                driver.compile(query);
-            driver.clear();
-            i++;
-        }
-        sw.close();
-        writeStringToFile(resultFile, sw);
-    }
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			if (query.toLowerCase().indexOf("create") >= 0
+					|| query.toLowerCase().indexOf("drop") >= 0
+					|| query.toLowerCase().indexOf("set") >= 0
+					|| query.toLowerCase().startsWith("\n\ncreate")
+					|| query.toLowerCase().startsWith("\n\ndrop")
+					|| query.toLowerCase().startsWith("\n\nset"))
+				driver.run(query);
+			else
+				driver.compile(query);
+			driver.clear();
+			i++;
+		}
+		sw.close();
+		writeStringToFile(resultFile, sw);
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestCase.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestCase.java
index 60a8b13..078de9a 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestCase.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestCase.java
@@ -15,133 +15,138 @@
 import edu.uci.ics.hivesterix.test.base.AbstractHivesterixTestCase;
 
 public class RuntimeFunctionTestCase extends AbstractHivesterixTestCase {
-    private File resultFile;
-    private FileSystem dfs;
+	private File resultFile;
+	private FileSystem dfs;
 
-    RuntimeFunctionTestCase(File queryFile, File resultFile) {
-        super("testRuntimeFunction", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+	RuntimeFunctionTestCase(File queryFile, File resultFile) {
+		super("testRuntimeFunction", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-    @Test
-    public void testRuntimeFunction() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	@Test
+	public void testRuntimeFunction() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
-        // Driver driver = new Driver(hconf);
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
+		// Driver driver = new Driver(hconf);
 
-        dfs = FileSystem.get(ConfUtil.getJobConf());
+		dfs = FileSystem.get(ConfUtil.getJobConf());
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            driver.run(query);
-            driver.clear();
-            i++;
-        }
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			driver.run(query);
+			driver.clear();
+			i++;
+		}
 
-        String warehouse = hconf.get("hive.metastore.warehouse.dir");
-        String tableName = removeExt(resultFile.getName());
-        String directory = warehouse + "/" + tableName + "/";
-        String localDirectory = "tmp";
+		String warehouse = hconf.get("hive.metastore.warehouse.dir");
+		String tableName = removeExt(resultFile.getName());
+		String directory = warehouse + "/" + tableName + "/";
+		String localDirectory = "tmp";
 
-        FileStatus[] files = dfs.listStatus(new Path(directory));
-        FileSystem lfs = null;
-        if (files == null) {
-            lfs = FileSystem.getLocal(ConfUtil.getJobConf());
-            files = lfs.listStatus(new Path(directory));
-        }
+		FileStatus[] files = dfs.listStatus(new Path(directory));
+		FileSystem lfs = null;
+		if (files == null) {
+			lfs = FileSystem.getLocal(ConfUtil.getJobConf());
+			files = lfs.listStatus(new Path(directory));
+		}
 
-        File resultDirectory = new File(localDirectory + "/" + tableName);
-        deleteDir(resultDirectory);
-        resultDirectory.mkdir();
+		File resultDirectory = new File(localDirectory + "/" + tableName);
+		deleteDir(resultDirectory);
+		resultDirectory.mkdir();
 
-        for (FileStatus fs : files) {
-            Path src = fs.getPath();
-            if (src.getName().indexOf("crc") >= 0)
-                continue;
+		for (FileStatus fs : files) {
+			Path src = fs.getPath();
+			if (src.getName().indexOf("crc") >= 0)
+				continue;
 
-            String destStr = localDirectory + "/" + tableName + "/" + src.getName();
-            Path dest = new Path(destStr);
-            if (lfs != null) {
-                lfs.copyToLocalFile(src, dest);
-                dfs.copyFromLocalFile(dest, new Path(directory));
-            } else
-                dfs.copyToLocalFile(src, dest);
-        }
+			String destStr = localDirectory + "/" + tableName + "/"
+					+ src.getName();
+			Path dest = new Path(destStr);
+			if (lfs != null) {
+				lfs.copyToLocalFile(src, dest);
+				dfs.copyFromLocalFile(dest, new Path(directory));
+			} else
+				dfs.copyToLocalFile(src, dest);
+		}
 
-        File[] rFiles = resultDirectory.listFiles();
-        StringBuilder sb = new StringBuilder();
-        for (File r : rFiles) {
-            if (r.getName().indexOf("crc") >= 0)
-                continue;
-            readFileToString(r, sb);
-        }
+		File[] rFiles = resultDirectory.listFiles();
+		StringBuilder sb = new StringBuilder();
+		for (File r : rFiles) {
+			if (r.getName().indexOf("crc") >= 0)
+				continue;
+			readFileToString(r, sb);
+		}
 
-        StringBuilder buf = new StringBuilder();
-        readFileToString(resultFile, buf);
-        StringBuffer errorMsg = new StringBuffer();
-        if (!equal(buf, sb, errorMsg)) {
-            throw new Exception("Result for " + queryFile + " changed:\n" + errorMsg.toString());
-        }
-        deleteDir(resultDirectory);
-    }
+		StringBuilder buf = new StringBuilder();
+		readFileToString(resultFile, buf);
+		StringBuffer errorMsg = new StringBuffer();
+		if (!equal(buf, sb, errorMsg)) {
+			throw new Exception("Result for " + queryFile + " changed:\n"
+					+ errorMsg.toString());
+		}
+		deleteDir(resultDirectory);
+	}
 
-    private void deleteDir(File resultDirectory) {
-        if (resultDirectory.exists()) {
-            File[] rFiles = resultDirectory.listFiles();
-            for (File r : rFiles)
-                r.delete();
-            resultDirectory.delete();
-        }
-    }
+	private void deleteDir(File resultDirectory) {
+		if (resultDirectory.exists()) {
+			File[] rFiles = resultDirectory.listFiles();
+			for (File r : rFiles)
+				r.delete();
+			resultDirectory.delete();
+		}
+	}
 
-    private boolean equal(StringBuilder sb1, StringBuilder sb2, StringBuffer errorMsg) {
-        String s1 = sb1.toString();
-        String s2 = sb2.toString();
-        String[] rowsOne = s1.split("\n");
-        String[] rowsTwo = s2.split("\n");
+	private boolean equal(StringBuilder sb1, StringBuilder sb2,
+			StringBuffer errorMsg) {
+		String s1 = sb1.toString();
+		String s2 = sb2.toString();
+		String[] rowsOne = s1.split("\n");
+		String[] rowsTwo = s2.split("\n");
 
-        if (rowsOne.length != rowsTwo.length)
-            return false;
+		if (rowsOne.length != rowsTwo.length)
+			return false;
 
-        for (int i = 0; i < rowsOne.length; i++) {
-            String row1 = rowsOne[i];
-            String row2 = rowsTwo[i];
+		for (int i = 0; i < rowsOne.length; i++) {
+			String row1 = rowsOne[i];
+			String row2 = rowsTwo[i];
 
-            if (row1.equals(row2))
-                continue;
+			if (row1.equals(row2))
+				continue;
 
-            String[] fields1 = row1.split("");
-            String[] fields2 = row2.split("");
+			String[] fields1 = row1.split("");
+			String[] fields2 = row2.split("");
 
-            for (int j = 0; j < fields1.length; j++) {
-                if (fields1[j].equals(fields2[j])) {
-                    continue;
-                } else if (fields1[j].indexOf('.') < 0) {
-                    errorMsg.append("line " + i + " column " + j + ": " + fields2[j] + " expected " + fields1[j]);
-                    return false;
-                } else {
-                    Float float1 = Float.parseFloat(fields1[j]);
-                    Float float2 = Float.parseFloat(fields2[j]);
+			for (int j = 0; j < fields1.length; j++) {
+				if (fields1[j].equals(fields2[j])) {
+					continue;
+				} else if (fields1[j].indexOf('.') < 0) {
+					errorMsg.append("line " + i + " column " + j + ": "
+							+ fields2[j] + " expected " + fields1[j]);
+					return false;
+				} else {
+					Float float1 = Float.parseFloat(fields1[j]);
+					Float float2 = Float.parseFloat(fields2[j]);
 
-                    if (Math.abs(float1 - float2) == 0)
-                        continue;
-                    else {
-                        errorMsg.append("line " + i + " column " + j + ": " + fields2[j] + " expected " + fields1[j]);
-                        return false;
-                    }
-                }
-            }
-        }
+					if (Math.abs(float1 - float2) == 0)
+						continue;
+					else {
+						errorMsg.append("line " + i + " column " + j + ": "
+								+ fields2[j] + " expected " + fields1[j]);
+						return false;
+					}
+				}
+			}
+		}
 
-        return true;
-    }
+		return true;
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
index 9610497..2093b1d 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuite.java
@@ -9,66 +9,66 @@
 

 public class RuntimeFunctionTestSuite extends AbstractTestSuiteClass {

 

-    private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";

-    private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";

-    private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";

+	private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";

+	private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";

+	private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";

 

-    private static final String FILE_EXTENSION_OF_RESULTS = "result";

+	private static final String FILE_EXTENSION_OF_RESULTS = "result";

 

-    public static Test suite() throws Exception {

-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

-        File testData = new File(PATH_TO_QUERIES);

-        File[] queries = testData.listFiles();

-        RuntimeFunctionTestSuite testSuite = new RuntimeFunctionTestSuite();

+	public static Test suite() throws Exception {

+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

+		File testData = new File(PATH_TO_QUERIES);

+		File[] queries = testData.listFiles();

+		RuntimeFunctionTestSuite testSuite = new RuntimeFunctionTestSuite();

 

-        // set hdfs and hyracks cluster, and load test data to hdfs

-        try {

-            testSuite.setup();

-            testSuite.loadData();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

+		// set hdfs and hyracks cluster, and load test data to hdfs

+		try {

+			testSuite.setup();

+			testSuite.loadData();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

 

-        for (File qFile : queries) {

-            if (isIgnored(qFile.getName(), ignores))

-                continue;

+		for (File qFile : queries) {

+			if (isIgnored(qFile.getName(), ignores))

+				continue;

 

-            if (qFile.isFile()) {

-                String resultFileName = hiveExtToResExt(qFile.getName());

-                File rFile = new File(PATH_TO_RESULTS + resultFileName);

-                testSuite.addTest(new RuntimeFunctionTestCase(qFile, rFile));

-            }

-        }

-        return testSuite;

-    }

+			if (qFile.isFile()) {

+				String resultFileName = hiveExtToResExt(qFile.getName());

+				File rFile = new File(PATH_TO_RESULTS + resultFileName);

+				testSuite.addTest(new RuntimeFunctionTestCase(qFile, rFile));

+			}

+		}

+		return testSuite;

+	}

 

-    private static String hiveExtToResExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

-    }

+	private static String hiveExtToResExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

+	}

 

-    /**

-     * Runs the tests and collects their result in a TestResult.

-     */

-    @Override

-    public void run(TestResult result) {

+	/**

+	 * Runs the tests and collects their result in a TestResult.

+	 */

+	@Override

+	public void run(TestResult result) {

 

-        int testCount = countTestCases();

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

-            Test each = this.testAt(i);

-            if (result.shouldStop())

-                break;

-            runTest(each, result);

-        }

+		int testCount = countTestCases();

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

+			Test each = this.testAt(i);

+			if (result.shouldStop())

+				break;

+			runTest(each, result);

+		}

 

-        // cleanup hdfs and hyracks cluster

-        try {

-            cleanup();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

-    }

+		// cleanup hdfs and hyracks cluster

+		try {

+			cleanup();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteCaseGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteCaseGenerator.java
index 9a90a33..1b45b41 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteCaseGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteCaseGenerator.java
@@ -14,86 +14,88 @@
 import edu.uci.ics.hivesterix.runtime.config.ConfUtil;
 import edu.uci.ics.hivesterix.test.base.AbstractHivesterixTestCase;
 
-public class RuntimeFunctionTestSuiteCaseGenerator extends AbstractHivesterixTestCase {
-    private File resultFile;
-    private FileSystem dfs;
-    
-    RuntimeFunctionTestSuiteCaseGenerator(File queryFile, File resultFile) {
-        super("testRuntimeFunction", queryFile);
-        this.queryFile = queryFile;
-        this.resultFile = resultFile;
-    }
+public class RuntimeFunctionTestSuiteCaseGenerator extends
+		AbstractHivesterixTestCase {
+	private File resultFile;
+	private FileSystem dfs;
 
-    @Test
-    public void testRuntimeFunction() throws Exception {
-        StringBuilder queryString = new StringBuilder();
-        readFileToString(queryFile, queryString);
-        String[] queries = queryString.toString().split(";");
-        StringWriter sw = new StringWriter();
+	RuntimeFunctionTestSuiteCaseGenerator(File queryFile, File resultFile) {
+		super("testRuntimeFunction", queryFile);
+		this.queryFile = queryFile;
+		this.resultFile = resultFile;
+	}
 
-        HiveConf hconf = ConfUtil.getHiveConf();
-        Driver driver = new Driver(hconf, new PrintWriter(sw));
-        driver.init();
+	@Test
+	public void testRuntimeFunction() throws Exception {
+		StringBuilder queryString = new StringBuilder();
+		readFileToString(queryFile, queryString);
+		String[] queries = queryString.toString().split(";");
+		StringWriter sw = new StringWriter();
 
-        dfs = FileSystem.get(ConfUtil.getJobConf());
+		HiveConf hconf = ConfUtil.getHiveConf();
+		Driver driver = new Driver(hconf, new PrintWriter(sw));
+		driver.init();
 
-        int i = 0;
-        for (String query : queries) {
-            if (i == queries.length - 1)
-                break;
-            driver.run(query);
-            driver.clear();
-            i++;
-        }
+		dfs = FileSystem.get(ConfUtil.getJobConf());
 
-        String warehouse = hconf.get("hive.metastore.warehouse.dir");
-        String tableName = removeExt(resultFile.getName());
-        String directory = warehouse + "/" + tableName + "/";
-        String localDirectory = "tmp";
+		int i = 0;
+		for (String query : queries) {
+			if (i == queries.length - 1)
+				break;
+			driver.run(query);
+			driver.clear();
+			i++;
+		}
 
-        FileStatus[] files = dfs.listStatus(new Path(directory));
-        FileSystem lfs = null;
-        if (files == null) {
-            lfs = FileSystem.getLocal(ConfUtil.getJobConf());
-            files = lfs.listStatus(new Path(directory));
-        }
+		String warehouse = hconf.get("hive.metastore.warehouse.dir");
+		String tableName = removeExt(resultFile.getName());
+		String directory = warehouse + "/" + tableName + "/";
+		String localDirectory = "tmp";
 
-        File resultDirectory = new File(localDirectory + "/" + tableName);
-        deleteDir(resultDirectory);
-        resultDirectory.mkdir();
+		FileStatus[] files = dfs.listStatus(new Path(directory));
+		FileSystem lfs = null;
+		if (files == null) {
+			lfs = FileSystem.getLocal(ConfUtil.getJobConf());
+			files = lfs.listStatus(new Path(directory));
+		}
 
-        for (FileStatus fs : files) {
-            Path src = fs.getPath();
-            if (src.getName().indexOf("crc") >= 0)
-                continue;
+		File resultDirectory = new File(localDirectory + "/" + tableName);
+		deleteDir(resultDirectory);
+		resultDirectory.mkdir();
 
-            String destStr = localDirectory + "/" + tableName + "/" + src.getName();
-            Path dest = new Path(destStr);
-            if (lfs != null) {
-                lfs.copyToLocalFile(src, dest);
-                dfs.copyFromLocalFile(dest, new Path(directory));
-            } else
-                dfs.copyToLocalFile(src, dest);
-        }
+		for (FileStatus fs : files) {
+			Path src = fs.getPath();
+			if (src.getName().indexOf("crc") >= 0)
+				continue;
 
-        File[] rFiles = resultDirectory.listFiles();
-        StringBuilder sb = new StringBuilder();
-        for (File r : rFiles) {
-            if (r.getName().indexOf("crc") >= 0)
-                continue;
-            readFileToString(r, sb);
-        }
-        deleteDir(resultDirectory);
+			String destStr = localDirectory + "/" + tableName + "/"
+					+ src.getName();
+			Path dest = new Path(destStr);
+			if (lfs != null) {
+				lfs.copyToLocalFile(src, dest);
+				dfs.copyFromLocalFile(dest, new Path(directory));
+			} else
+				dfs.copyToLocalFile(src, dest);
+		}
 
-        writeStringToFile(resultFile, sb);
-    }
+		File[] rFiles = resultDirectory.listFiles();
+		StringBuilder sb = new StringBuilder();
+		for (File r : rFiles) {
+			if (r.getName().indexOf("crc") >= 0)
+				continue;
+			readFileToString(r, sb);
+		}
+		deleteDir(resultDirectory);
 
-    private void deleteDir(File resultDirectory) {
-        if (resultDirectory.exists()) {
-            File[] rFiles = resultDirectory.listFiles();
-            for (File r : rFiles)
-                r.delete();
-            resultDirectory.delete();
-        }
-    }
+		writeStringToFile(resultFile, sb);
+	}
+
+	private void deleteDir(File resultDirectory) {
+		if (resultDirectory.exists()) {
+			File[] rFiles = resultDirectory.listFiles();
+			for (File r : rFiles)
+				r.delete();
+			resultDirectory.delete();
+		}
+	}
 }
diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteGenerator.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteGenerator.java
index ca2bd6d..a67f475 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteGenerator.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/runtimefunction/RuntimeFunctionTestSuiteGenerator.java
@@ -9,66 +9,67 @@
 

 public class RuntimeFunctionTestSuiteGenerator extends AbstractTestSuiteClass {

 

-    private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";

-    private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";

-    private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";

+	private static final String PATH_TO_QUERIES = "src/test/resources/runtimefunctionts/queries/";

+	private static final String PATH_TO_RESULTS = "src/test/resources/runtimefunctionts/results/";

+	private static final String PATH_TO_IGNORES = "src/test/resources/runtimefunctionts/ignore.txt";

 

-    private static final String FILE_EXTENSION_OF_RESULTS = "result";

+	private static final String FILE_EXTENSION_OF_RESULTS = "result";

 

-    public static Test suite() throws Exception {

-        List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

-        File testData = new File(PATH_TO_QUERIES);

-        File[] queries = testData.listFiles();

-        RuntimeFunctionTestSuiteGenerator testSuite = new RuntimeFunctionTestSuiteGenerator();

+	public static Test suite() throws Exception {

+		List<String> ignores = getIgnoreList(PATH_TO_IGNORES);

+		File testData = new File(PATH_TO_QUERIES);

+		File[] queries = testData.listFiles();

+		RuntimeFunctionTestSuiteGenerator testSuite = new RuntimeFunctionTestSuiteGenerator();

 

-        // set hdfs and hyracks cluster, and load test data to hdfs

-        try {

-            testSuite.setup();

-            testSuite.loadData();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

+		// set hdfs and hyracks cluster, and load test data to hdfs

+		try {

+			testSuite.setup();

+			testSuite.loadData();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

 

-        for (File qFile : queries) {

-            if (isIgnored(qFile.getName(), ignores))

-                continue;

+		for (File qFile : queries) {

+			if (isIgnored(qFile.getName(), ignores))

+				continue;

 

-            if (qFile.isFile() && qFile.getName().startsWith("q16_")) {

-                String resultFileName = hiveExtToResExt(qFile.getName());

-                File rFile = new File(PATH_TO_RESULTS + resultFileName);

-                testSuite.addTest(new RuntimeFunctionTestSuiteCaseGenerator(qFile, rFile));

-            }

-        }

-        return testSuite;

-    }

+			if (qFile.isFile() && qFile.getName().startsWith("q16_")) {

+				String resultFileName = hiveExtToResExt(qFile.getName());

+				File rFile = new File(PATH_TO_RESULTS + resultFileName);

+				testSuite.addTest(new RuntimeFunctionTestSuiteCaseGenerator(

+						qFile, rFile));

+			}

+		}

+		return testSuite;

+	}

 

-    private static String hiveExtToResExt(String fname) {

-        int dot = fname.lastIndexOf('.');

-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

-    }

+	private static String hiveExtToResExt(String fname) {

+		int dot = fname.lastIndexOf('.');

+		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;

+	}

 

-    /**

-     * Runs the tests and collects their result in a TestResult.

-     */

-    @Override

-    public void run(TestResult result) {

+	/**

+	 * Runs the tests and collects their result in a TestResult.

+	 */

+	@Override

+	public void run(TestResult result) {

 

-        int testCount = countTestCases();

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

-            Test each = this.testAt(i);

-            if (result.shouldStop())

-                break;

-            runTest(each, result);

-        }

+		int testCount = countTestCases();

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

+			Test each = this.testAt(i);

+			if (result.shouldStop())

+				break;

+			runTest(each, result);

+		}

 

-        // cleanup hdfs and hyracks cluster

-        try {

-            cleanup();

-        } catch (Exception e) {

-            e.printStackTrace();

-            throw new IllegalStateException(e.getMessage());

-        }

-    }

+		// cleanup hdfs and hyracks cluster

+		try {

+			cleanup();

+		} catch (Exception e) {

+			e.printStackTrace();

+			throw new IllegalStateException(e.getMessage());

+		}

+	}

 

 }

diff --git a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/serde/SerDeTest.java b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/serde/SerDeTest.java
index 576d648..b5db432 100644
--- a/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/serde/SerDeTest.java
+++ b/hivesterix/src/test/java/edu/uci/ics/hivesterix/test/serde/SerDeTest.java
@@ -45,174 +45,188 @@
  * TestLazySimpleSerDe.
  * 
  */
+@SuppressWarnings({ "deprecation", "rawtypes" })
 public class SerDeTest extends TestCase {
 
-    /**
-     * Test the LazySimpleSerDe class.
-     */
-    public void testLazySimpleSerDe() throws Throwable {
-        try {
-            // Create the SerDe
-            LazySimpleSerDe serDe = new LazySimpleSerDe();
-            Configuration conf = new Configuration();
-            Properties tbl = createProperties();
-            serDe.initialize(conf, tbl);
+	/**
+	 * Test the LazySimpleSerDe class.
+	 */
+	public void testLazySimpleSerDe() throws Throwable {
+		try {
+			// Create the SerDe
+			LazySimpleSerDe serDe = new LazySimpleSerDe();
+			Configuration conf = new Configuration();
+			Properties tbl = createProperties();
+			serDe.initialize(conf, tbl);
 
-            LazySerDe outputSerde = new LazySerDe();
-            outputSerde.initialize(conf, tbl);
+			LazySerDe outputSerde = new LazySerDe();
+			outputSerde.initialize(conf, tbl);
 
-            // Data
-            Text t = new Text("123\t456\t789\t1000\t5.3\thive and hadoop\t1.\tNULL");
-            String s = "123\t456\t789\t1000\t5.3\thive and hadoop\t1\tqf";
+			// Data
+			String s = "123\t456\t789\t1000\t5.3\thive and hadoop\t1\tqf";
 
-            byte[] bytes = s.getBytes();
-            Writable bytesWritable = new BytesWritable(bytes);
+			byte[] bytes = s.getBytes();
+			Writable bytesWritable = new BytesWritable(bytes);
 
-            // Test
-            // deserializeAndSerialize(serDe, t, s, expectedFieldsData);
-            Object row = serDe.deserialize(bytesWritable); // test my serde
-            StructObjectInspector simpleInspector = (StructObjectInspector) serDe.getObjectInspector();
-            List<Object> fields = simpleInspector.getStructFieldsDataAsList(row);
-            List<? extends StructField> fieldRefs = simpleInspector.getAllStructFieldRefs();
+			// Test
+			// deserializeAndSerialize(serDe, t, s, expectedFieldsData);
+			Object row = serDe.deserialize(bytesWritable); // test my serde
+			StructObjectInspector simpleInspector = (StructObjectInspector) serDe
+					.getObjectInspector();
+			List<Object> fields = simpleInspector
+					.getStructFieldsDataAsList(row);
+			List<? extends StructField> fieldRefs = simpleInspector
+					.getAllStructFieldRefs();
 
-            int i = 0;
-            for (Object field : fields) {
-                BytesWritable fieldWritable = (BytesWritable) outputSerde.serialize(field, fieldRefs.get(i)
-                        .getFieldObjectInspector());
-                System.out.print(fieldWritable.getSize() + "|");
-                i++;
-            }
+			int i = 0;
+			for (Object field : fields) {
+				BytesWritable fieldWritable = (BytesWritable) outputSerde
+						.serialize(field, fieldRefs.get(i)
+								.getFieldObjectInspector());
+				System.out.print(fieldWritable.getSize() + "|");
+				i++;
+			}
 
-            // Writable output = outputSerde.serialize(row, serDe
-            // .getObjectInspector());
-            // System.out.println(output);
-            //
-            // Object row2 = outputSerde.deserialize(output);
-            // Writable output2 = serDe.serialize(row2, outputSerde
-            // .getObjectInspector());
-            // System.out.println(output2);
+			// Writable output = outputSerde.serialize(row, serDe
+			// .getObjectInspector());
+			// System.out.println(output);
+			//
+			// Object row2 = outputSerde.deserialize(output);
+			// Writable output2 = serDe.serialize(row2, outputSerde
+			// .getObjectInspector());
+			// System.out.println(output2);
 
-            // System.out.println(output);
-            // deserializeAndSerialize(outputSerde, t, s, expectedFieldsData);
+			// System.out.println(output);
+			// deserializeAndSerialize(outputSerde, t, s, expectedFieldsData);
 
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
+		} catch (Throwable e) {
+			e.printStackTrace();
+			throw e;
+		}
+	}
 
-    private void deserializeAndSerialize(SerDe serDe, Text t, String s, Object[] expectedFieldsData)
-            throws SerDeException {
-        // Get the row structure
-        StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector();
-        List<? extends StructField> fieldRefs = oi.getAllStructFieldRefs();
-        assertEquals(8, fieldRefs.size());
+	private void deserializeAndSerialize(SerDe serDe, Text t, String s,
+			Object[] expectedFieldsData) throws SerDeException {
+		// Get the row structure
+		StructObjectInspector oi = (StructObjectInspector) serDe
+				.getObjectInspector();
+		List<? extends StructField> fieldRefs = oi.getAllStructFieldRefs();
+		assertEquals(8, fieldRefs.size());
 
-        // Deserialize
-        Object row = serDe.deserialize(t);
-        for (int i = 0; i < fieldRefs.size(); i++) {
-            Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
-            if (fieldData != null) {
-                fieldData = ((LazyPrimitive) fieldData).getWritableObject();
-            }
-            assertEquals("Field " + i, expectedFieldsData[i], fieldData);
-        }
-        // Serialize
-        assertEquals(Text.class, serDe.getSerializedClass());
-        Text serializedText = (Text) serDe.serialize(row, oi);
-        assertEquals("Serialized data", s, serializedText.toString());
-    }
+		// Deserialize
+		Object row = serDe.deserialize(t);
+		for (int i = 0; i < fieldRefs.size(); i++) {
+			Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
+			if (fieldData != null) {
+				fieldData = ((LazyPrimitive) fieldData).getWritableObject();
+			}
+			assertEquals("Field " + i, expectedFieldsData[i], fieldData);
+		}
+		// Serialize
+		assertEquals(Text.class, serDe.getSerializedClass());
+		Text serializedText = (Text) serDe.serialize(row, oi);
+		assertEquals("Serialized data", s, serializedText.toString());
+	}
 
-    private Properties createProperties() {
-        Properties tbl = new Properties();
+	private Properties createProperties() {
+		Properties tbl = new Properties();
 
-        // Set the configuration parameters
-        tbl.setProperty(Constants.SERIALIZATION_FORMAT, "9");
-        tbl.setProperty("columns", "abyte,ashort,aint,along,adouble,astring,anullint,anullstring");
-        tbl.setProperty("columns.types", "tinyint:smallint:int:bigint:double:string:int:string");
-        tbl.setProperty(Constants.SERIALIZATION_NULL_FORMAT, "NULL");
-        return tbl;
-    }
+		// Set the configuration parameters
+		tbl.setProperty(Constants.SERIALIZATION_FORMAT, "9");
+		tbl.setProperty("columns",
+				"abyte,ashort,aint,along,adouble,astring,anullint,anullstring");
+		tbl.setProperty("columns.types",
+				"tinyint:smallint:int:bigint:double:string:int:string");
+		tbl.setProperty(Constants.SERIALIZATION_NULL_FORMAT, "NULL");
+		return tbl;
+	}
 
-    /**
-     * Test the LazySimpleSerDe class with LastColumnTakesRest option.
-     */
-    public void testLazySimpleSerDeLastColumnTakesRest() throws Throwable {
-        try {
-            // Create the SerDe
-            LazySimpleSerDe serDe = new LazySimpleSerDe();
-            Configuration conf = new Configuration();
-            Properties tbl = createProperties();
-            tbl.setProperty(Constants.SERIALIZATION_LAST_COLUMN_TAKES_REST, "true");
-            serDe.initialize(conf, tbl);
+	/**
+	 * Test the LazySimpleSerDe class with LastColumnTakesRest option.
+	 */
+	public void testLazySimpleSerDeLastColumnTakesRest() throws Throwable {
+		try {
+			// Create the SerDe
+			LazySimpleSerDe serDe = new LazySimpleSerDe();
+			Configuration conf = new Configuration();
+			Properties tbl = createProperties();
+			tbl.setProperty(Constants.SERIALIZATION_LAST_COLUMN_TAKES_REST,
+					"true");
+			serDe.initialize(conf, tbl);
 
-            // Data
-            Text t = new Text("123\t456\t789\t1000\t5.3\thive and hadoop\t1.\ta\tb\t");
-            String s = "123\t456\t789\t1000\t5.3\thive and hadoop\tNULL\ta\tb\t";
-            Object[] expectedFieldsData = { new ByteWritable((byte) 123), new ShortWritable((short) 456),
-                    new IntWritable(789), new LongWritable(1000), new DoubleWritable(5.3), new Text("hive and hadoop"),
-                    null, new Text("a\tb\t") };
+			// Data
+			Text t = new Text(
+					"123\t456\t789\t1000\t5.3\thive and hadoop\t1.\ta\tb\t");
+			String s = "123\t456\t789\t1000\t5.3\thive and hadoop\tNULL\ta\tb\t";
+			Object[] expectedFieldsData = { new ByteWritable((byte) 123),
+					new ShortWritable((short) 456), new IntWritable(789),
+					new LongWritable(1000), new DoubleWritable(5.3),
+					new Text("hive and hadoop"), null, new Text("a\tb\t") };
 
-            // Test
-            deserializeAndSerialize(serDe, t, s, expectedFieldsData);
+			// Test
+			deserializeAndSerialize(serDe, t, s, expectedFieldsData);
 
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
+		} catch (Throwable e) {
+			e.printStackTrace();
+			throw e;
+		}
+	}
 
-    /**
-     * Test the LazySimpleSerDe class with extra columns.
-     */
-    public void testLazySimpleSerDeExtraColumns() throws Throwable {
-        try {
-            // Create the SerDe
-            LazySimpleSerDe serDe = new LazySimpleSerDe();
-            Configuration conf = new Configuration();
-            Properties tbl = createProperties();
-            serDe.initialize(conf, tbl);
+	/**
+	 * Test the LazySimpleSerDe class with extra columns.
+	 */
+	public void testLazySimpleSerDeExtraColumns() throws Throwable {
+		try {
+			// Create the SerDe
+			LazySimpleSerDe serDe = new LazySimpleSerDe();
+			Configuration conf = new Configuration();
+			Properties tbl = createProperties();
+			serDe.initialize(conf, tbl);
 
-            // Data
-            Text t = new Text("123\t456\t789\t1000\t5.3\thive and hadoop\t1.\ta\tb\t");
-            String s = "123\t456\t789\t1000\t5.3\thive and hadoop\tNULL\ta";
-            Object[] expectedFieldsData = { new ByteWritable((byte) 123), new ShortWritable((short) 456),
-                    new IntWritable(789), new LongWritable(1000), new DoubleWritable(5.3), new Text("hive and hadoop"),
-                    null, new Text("a") };
+			// Data
+			Text t = new Text(
+					"123\t456\t789\t1000\t5.3\thive and hadoop\t1.\ta\tb\t");
+			String s = "123\t456\t789\t1000\t5.3\thive and hadoop\tNULL\ta";
+			Object[] expectedFieldsData = { new ByteWritable((byte) 123),
+					new ShortWritable((short) 456), new IntWritable(789),
+					new LongWritable(1000), new DoubleWritable(5.3),
+					new Text("hive and hadoop"), null, new Text("a") };
 
-            // Test
-            deserializeAndSerialize(serDe, t, s, expectedFieldsData);
+			// Test
+			deserializeAndSerialize(serDe, t, s, expectedFieldsData);
 
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
+		} catch (Throwable e) {
+			e.printStackTrace();
+			throw e;
+		}
+	}
 
-    /**
-     * Test the LazySimpleSerDe class with missing columns.
-     */
-    public void testLazySimpleSerDeMissingColumns() throws Throwable {
-        try {
-            // Create the SerDe
-            LazySimpleSerDe serDe = new LazySimpleSerDe();
-            Configuration conf = new Configuration();
-            Properties tbl = createProperties();
-            serDe.initialize(conf, tbl);
+	/**
+	 * Test the LazySimpleSerDe class with missing columns.
+	 */
+	public void testLazySimpleSerDeMissingColumns() throws Throwable {
+		try {
+			// Create the SerDe
+			LazySimpleSerDe serDe = new LazySimpleSerDe();
+			Configuration conf = new Configuration();
+			Properties tbl = createProperties();
+			serDe.initialize(conf, tbl);
 
-            // Data
-            Text t = new Text("123\t456\t789\t1000\t5.3\t");
-            String s = "123\t456\t789\t1000\t5.3\t\tNULL\tNULL";
-            Object[] expectedFieldsData = { new ByteWritable((byte) 123), new ShortWritable((short) 456),
-                    new IntWritable(789), new LongWritable(1000), new DoubleWritable(5.3), new Text(""), null, null };
+			// Data
+			Text t = new Text("123\t456\t789\t1000\t5.3\t");
+			String s = "123\t456\t789\t1000\t5.3\t\tNULL\tNULL";
+			Object[] expectedFieldsData = { new ByteWritable((byte) 123),
+					new ShortWritable((short) 456), new IntWritable(789),
+					new LongWritable(1000), new DoubleWritable(5.3),
+					new Text(""), null, null };
 
-            // Test
-            deserializeAndSerialize(serDe, t, s, expectedFieldsData);
+			// Test
+			deserializeAndSerialize(serDe, t, s, expectedFieldsData);
 
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
+		} catch (Throwable e) {
+			e.printStackTrace();
+			throw e;
+		}
+	}
 
 }
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-cli/pom.xml
index 3933be0..7bb6123 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-cli/pom.xml
@@ -41,6 +41,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
new file mode 100644
index 0000000..c90ce5a
--- /dev/null
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+/**
+ * This class provides the raw bytes-based comparison and hash function for UTF8 strings.
+ * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
+ * But it works for single-byte character languages.
+ */
+public final class RawUTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new RawUTF8StringPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        int utflen1 = UTF8StringPointable.getUTFLength(this.bytes, this.start);
+        int utflen2 = UTF8StringPointable.getUTFLength(bytes, start);
+
+        int c1 = 0;
+        int c2 = 0;
+
+        int s1Start = this.start + 2;
+        int s2Start = start + 2;
+
+        while (c1 < utflen1 && c2 < utflen2) {
+            char ch1 = (char) this.bytes[s1Start + c1];
+            char ch2 = (char) bytes[s2Start + c2];
+
+            if (ch1 != ch2) {
+                return ch1 - ch2;
+            }
+            c1++;
+            c2++;
+        }
+        return utflen1 - utflen2;
+    }
+
+    @Override
+    public int hash() {
+        int h = 0;
+        int utflen = UTF8StringPointable.getUTFLength(bytes, start);
+        int sStart = start + 2;
+        int c = 0;
+
+        while (c < utflen) {
+            char ch = (char) bytes[sStart + c];
+            h = 31 * h + ch;
+            c++;
+        }
+        return h;
+    }
+
+    public void toString(StringBuilder buffer) {
+        UTF8StringPointable.toString(buffer, bytes, start);
+    }
+}
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
index f6d6093..866ebb0 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -216,4 +216,4 @@
     public void toString(StringBuilder buffer) {
         toString(buffer, bytes, start);
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
index e8dc9b4..7d7d2c1 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
@@ -1,34 +1,28 @@
 package edu.uci.ics.hyracks.data.std.util;
 
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.api.IMutableValueStorage;
 import edu.uci.ics.hyracks.data.std.api.IValueReference;
 
 public class ArrayBackedValueStorage implements IMutableValueStorage {
-    private final ByteArrayAccessibleOutputStream baaos;
-    private final DataOutputStream dos;
-
-    public ArrayBackedValueStorage() {
-        baaos = new ByteArrayAccessibleOutputStream();
-        dos = new DataOutputStream(baaos);
-    }
+   
+    private final GrowableArray data = new GrowableArray();
 
     @Override
     public void reset() {
-        baaos.reset();
+        data.reset();
     }
 
     @Override
     public DataOutput getDataOutput() {
-        return dos;
+        return data.getDataOutput();
     }
 
     @Override
     public byte[] getByteArray() {
-        return baaos.getByteArray();
+        return data.getByteArray();
     }
 
     @Override
@@ -38,12 +32,12 @@
 
     @Override
     public int getLength() {
-        return baaos.size();
+        return data.getLength();
     }
 
     public void append(IValueReference value) {
         try {
-            dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+            data.append(value);
         } catch (IOException e) {
             e.printStackTrace();
         }
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
new file mode 100644
index 0000000..c174d4e
--- /dev/null
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.data.std.util;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
+
+public class GrowableArray implements IDataOutputProvider {
+    private final ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+    private final DataOutputStream dos = new DataOutputStream(baaos);
+
+    @Override
+    public DataOutput getDataOutput() {
+        return dos;
+    }
+
+    public void reset() {
+        baaos.reset();
+    }
+
+    public byte[] getByteArray() {
+        return baaos.getByteArray();
+    }
+
+    public int getLength() {
+        return baaos.size();
+    }
+
+    public void append(IValueReference value) throws IOException {
+        dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index 989bc6b..8c865c4 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -15,14 +15,13 @@
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 /**
  * Array backed tuple builder.
@@ -30,25 +29,21 @@
  * @author vinayakb
  */
 public class ArrayTupleBuilder implements IDataOutputProvider {
-    private final ByteArrayAccessibleOutputStream baaos;
-    private final DataOutputStream dos;
+    private final GrowableArray fieldData = new GrowableArray();
     private final int[] fEndOffsets;
     private int nextField;
 
     public ArrayTupleBuilder(int nFields) {
-        baaos = new ByteArrayAccessibleOutputStream();
-        dos = new DataOutputStream(baaos);
         fEndOffsets = new int[nFields];
     }
 
     /**
      * Resets the builder.
-     * 
      * reset() must be called before attempting to create a new tuple.
      */
     public void reset() {
         nextField = 0;
-        baaos.reset();
+        fieldData.reset();
     }
 
     /**
@@ -66,7 +61,7 @@
      * @return Data byte array.
      */
     public byte[] getByteArray() {
-        return baaos.getByteArray();
+        return fieldData.getByteArray();
     }
 
     /**
@@ -75,7 +70,7 @@
      * @return data area size.
      */
     public int getSize() {
-        return baaos.size();
+        return fieldData.getLength();
     }
 
     /**
@@ -96,14 +91,15 @@
         int fStartOffset = accessor.getFieldStartOffset(tIndex, fIndex);
         int fLen = accessor.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
         try {
-            dos.write(accessor.getBuffer().array(), startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
+            fieldData.getDataOutput().write(accessor.getBuffer().array(),
+                    startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
             if (FrameConstants.DEBUG_FRAME_IO) {
-                dos.writeInt(FrameConstants.FRAME_FIELD_MAGIC);
+                fieldData.getDataOutput().writeInt(FrameConstants.FRAME_FIELD_MAGIC);
             }
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -117,8 +113,8 @@
      * @throws HyracksDataException
      */
     public <T> void addField(ISerializerDeserializer<T> serDeser, T instance) throws HyracksDataException {
-        serDeser.serialize(instance, dos);
-        fEndOffsets[nextField++] = baaos.size();
+        serDeser.serialize(instance, fieldData.getDataOutput());
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -134,11 +130,11 @@
      */
     public void addField(byte[] bytes, int start, int length) throws HyracksDataException {
         try {
-            dos.write(bytes, start, length);
+            fieldData.getDataOutput().write(bytes, start, length);
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -146,7 +142,14 @@
      */
     @Override
     public DataOutput getDataOutput() {
-        return dos;
+        return fieldData.getDataOutput();
+    }
+
+    /**
+     * Get the growable array storing the field data.
+     */
+    public GrowableArray getFieldData() {
+        return fieldData;
     }
 
     /**
@@ -156,6 +159,6 @@
      * data.
      */
     public void addFieldEndOffset() {
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..e95e9c2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class DoubleNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+                if (prefix >= 0) {
+                    return prefix ^ Integer.MIN_VALUE;
+                } else {
+                    return (int) ((long) 0xffffffff - (long) prefix);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..d58afc1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class FloatNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+                if (prefix >= 0) {
+                    return prefix ^ Integer.MIN_VALUE;
+                } else {
+                    return (int) ((long) 0xffffffff - (long) prefix);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..4589909
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+
+public class Integer64NormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 8735044913496854551L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+            private static final int POSTIVE_LONG_MASK = (3 << 30);
+            private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+            private static final int NEGATIVE_LONG_MASK = (0 << 30);
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                long value = Integer64SerializerDeserializer.getLong(bytes, start);
+                int highValue = (int) (value >> 32);
+                if (highValue > 0) {
+                    /**
+                     * larger than Integer.MAX
+                     */
+                    int highNmk = getKey(highValue);
+                    highNmk >>= 2;
+                    highNmk |= POSTIVE_LONG_MASK;
+                    return highNmk;
+                } else if (highValue == 0) {
+                    /**
+                     * smaller than Integer.MAX but >=0
+                     */
+                    int lowNmk = (int) value;
+                    lowNmk >>= 2;
+                    lowNmk |= NON_NEGATIVE_INT_MASK;
+                    return lowNmk;
+                } else {
+                    /**
+                     * less than 0: have not optimized for that
+                     */
+                    int highNmk = getKey(highValue);
+                    highNmk >>= 2;
+                    highNmk |= NEGATIVE_LONG_MASK;
+                    return highNmk;
+                }
+            }
+
+            private int getKey(int value) {
+                return value ^ Integer.MIN_VALUE;
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
index 2f7a778..6a01842 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
@@ -27,8 +27,7 @@
             @Override
             public int normalize(byte[] bytes, int start, int length) {
                 int value = IntegerSerializerDeserializer.getInt(bytes, start);
-                long unsignedValue = (long) value;
-                return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+                return value ^Integer.MIN_VALUE;
             }
         };
     }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
index 6a9dab7..d9191c7 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
@@ -32,7 +32,7 @@
         return new IValueParser() {
             @Override
             public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
-                long n = 0;
+                int n = 0;
                 int sign = 1;
                 int i = 0;
                 boolean pre = true;
@@ -102,7 +102,7 @@
                             throw new HyracksDataException("Encountered " + ch);
                     }
                 }
-
+                
                 try {
                     out.writeLong(n * sign);
                 } catch (IOException e) {
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
index a29209c..8a30a71 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
@@ -18,17 +18,19 @@
 import java.io.IOException;
 
 public class StringUtils {
-    public static void writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
-
+    public static int writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
         if (c >= 0x0000 && c <= 0x007F) {
             dos.writeByte(c);
+            return 1;
         } else if (c <= 0x07FF) {
             dos.writeByte((byte) (0xC0 | ((c >> 6) & 0x3F)));
             dos.writeByte((byte) (0x80 | (c & 0x3F)));
+            return 2;
         } else {
             dos.writeByte((byte) (0xE0 | ((c >> 12) & 0x0F)));
             dos.writeByte((byte) (0x80 | ((c >> 6) & 0x3F)));
             dos.writeByte((byte) (0x80 | (c & 0x3F)));
+            return 3;
         }
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 7e84229..6870e71 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -14,15 +14,18 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.join;
 
+import java.io.DataOutput;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -42,9 +45,12 @@
     private RunFileReader runFileReader;
     private int currentMemSize = 0;
     private final RunFileWriter runFileWriter;
+    private final boolean isLeftOuter;
+    private final ArrayTupleBuilder nullTupleBuilder;
 
     public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
-            ITuplePairComparator comparators, int memSize) throws HyracksDataException {
+            ITuplePairComparator comparators, int memSize, boolean isLeftOuter, INullWriter[] nullWriters1)
+            throws HyracksDataException {
         this.accessorInner = accessor1;
         this.accessorOuter = accessor0;
         this.appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -56,6 +62,19 @@
         this.memSize = memSize;
         this.ctx = ctx;
 
+        this.isLeftOuter = isLeftOuter;
+        if (isLeftOuter) {
+            int innerFieldCount = accessorInner.getFieldCount();
+            nullTupleBuilder = new ArrayTupleBuilder(innerFieldCount);
+            DataOutput out = nullTupleBuilder.getDataOutput();
+            for (int i = 0; i < innerFieldCount; i++) {
+                nullWriters1[i].writeNull(out);
+                nullTupleBuilder.addFieldEndOffset();
+            }
+        } else {
+            nullTupleBuilder = null;
+        }
+
         FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                 this.getClass().getSimpleName() + this.toString());
         runFileWriter = new RunFileWriter(file, ctx.getIOManager());
@@ -108,9 +127,11 @@
         int tupleCount1 = accessorInner.getTupleCount();
 
         for (int i = 0; i < tupleCount0; ++i) {
+            boolean matchFound = false;
             for (int j = 0; j < tupleCount1; ++j) {
                 int c = compare(accessorOuter, i, accessorInner, j);
                 if (c == 0) {
+                    matchFound = true;
                     if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
                         flushFrame(outBuffer, writer);
                         appender.reset(outBuffer, true);
@@ -120,6 +141,18 @@
                     }
                 }
             }
+
+            if (!matchFound && isLeftOuter) {
+                if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+                        nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+                    flushFrame(outBuffer, writer);
+                    appender.reset(outBuffer, true);
+                    if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+                            nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+                        throw new IllegalStateException();
+                    }
+                }
+            }
         }
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index a699703..0be01c1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -25,6 +25,8 @@
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
@@ -47,13 +49,18 @@
     private static final long serialVersionUID = 1L;
     private final ITuplePairComparatorFactory comparatorFactory;
     private final int memSize;
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
 
     public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize) {
+            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
+            boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.comparatorFactory = comparatorFactory;
         this.recordDescriptors[0] = recordDescriptor;
         this.memSize = memSize;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
     }
 
     @Override
@@ -111,6 +118,13 @@
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
 
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private JoinCacheTaskState state;
 
@@ -118,8 +132,11 @@
                 public void open() throws HyracksDataException {
                     state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
                             partition));
+
                     state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
-                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize);
+                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, isLeftOuter,
+                            nullWriters1);
+
                 }
 
                 @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 2905574..6e2b16a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -81,6 +81,8 @@
 
     private int[] buildPSizeInFrames; //Used for partition tuning
     private int freeFramesCounter; //Used for partition tuning
+    
+    private boolean isTableEmpty;	//Added for handling the case, where build side is empty (tableSize is 0)
 
     public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
             String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
@@ -89,10 +91,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = probeHpc;
-        this.probeHpc = buildHpc;
-        this.buildKeys = keys0;
-        this.probeKeys = keys1;
+        this.buildHpc = buildHpc; 	
+        this.probeHpc = probeHpc; 	
+        this.buildKeys = keys1; 	
+        this.probeKeys = keys0;		
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -117,10 +119,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = probeHpc;
-        this.probeHpc = buildHpc;
-        this.buildKeys = keys0;
-        this.probeKeys = keys1;
+        this.buildHpc = buildHpc; 	
+        this.probeHpc = probeHpc; 	
+        this.buildKeys = keys1; 	
+        this.probeKeys = keys0;		
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -171,6 +173,12 @@
     public void build(ByteBuffer buffer) throws HyracksDataException {
         accessorBuild.reset(buffer);
         int tupleCount = accessorBuild.getTupleCount();
+   
+        boolean print = false;
+    	if(print){
+    		accessorBuild.prettyPrint();
+    	}
+        
         for (int i = 0; i < tupleCount; ++i) {
             int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
             processTuple(i, pid);
@@ -338,6 +346,7 @@
 
         createInMemoryJoiner(inMemTupCount);
         cacheInMemJoin();
+        this.isTableEmpty = (inMemTupCount == 0);
     }
 
     private void partitionTune() throws HyracksDataException {
@@ -457,10 +466,14 @@
     }
 
     public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-
         accessorProbe.reset(buffer);
         int tupleCount = accessorProbe.getTupleCount();
 
+        boolean print = false;
+    	if(print){
+    		accessorProbe.prettyPrint();
+    	}
+        
         if (numOfSpilledParts == 0) {
             inMemJoiner.join(buffer, writer);
             return;
@@ -604,4 +617,8 @@
                 + freeFramesCounter;
         return s;
     }
+    
+    public boolean isTableEmpty(){
+    	return this.isTableEmpty;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 3a7ee2c..19479df 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -167,10 +167,10 @@
         ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(probeAid, buildAid);
 
         builder.addActivity(this, phase1);
-        builder.addSourceEdge(0, phase1, 0);
+        builder.addSourceEdge(1, phase1, 0);
 
         builder.addActivity(this, phase2);
-        builder.addSourceEdge(1, phase2, 0);
+        builder.addSourceEdge(0, phase2, 0);
 
         builder.addBlockingEdge(phase1, phase2);
 
@@ -253,14 +253,7 @@
             for (int i = 0; i < comparatorFactories.length; i++) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
-
-            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
-            if (isLeftOuter) {
-                for (int i = 0; i < nullWriterFactories1.length; i++) {
-                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
-                }
-            }
-
+            
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
                         .getJobId(), new TaskId(getActivityId(), partition));
@@ -278,9 +271,17 @@
                     state.memForJoin = memsize - 2;
                     state.numOfPartitions = getNumberOfPartitions(state.memForJoin, inputsize0, fudgeFactor,
                             nPartitions);
-                    state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
-                            PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
-                            buildHpc);
+                    if(!isLeftOuter){
+                    	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                                buildHpc);
+                    }
+                    else{
+                    	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                                buildHpc, isLeftOuter, nullWriterFactories1);
+                    }
+                    
                     state.hybridHJ.initBuild();
                 }
 
@@ -368,7 +369,9 @@
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    state.hybridHJ.probe(buffer, writer);
+                	if(!state.hybridHJ.isTableEmpty()){
+                		state.hybridHJ.probe(buffer, writer);
+                	}
                 }
 
                 @Override
@@ -604,7 +607,7 @@
                         throws HyracksDataException {
 
                     NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
-                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize);
+                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize, false, null);
 
                     ByteBuffer cacheBuff = ctx.allocateFrame();
                     innerReader.open();
diff --git a/hyracks/hyracks-dist/pom.xml b/hyracks/hyracks-dist/pom.xml
new file mode 100755
index 0000000..c86c4e6
--- /dev/null
+++ b/hyracks/hyracks-dist/pom.xml
@@ -0,0 +1,74 @@
+<?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>hyracks</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>hyracks-dist</artifactId>
+	<name>hyracks-dist</name>
+
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.6</source>
+					<target>1.6</target>
+				</configuration>
+			</plugin>
+			<plugin>
+				<artifactId>maven-resources-plugin</artifactId>
+				<version>2.5</version>
+				<executions>
+					<execution>
+						<id>copy-scripts</id>
+						<!-- here the phase you need -->
+						<phase>package</phase>
+						<goals>
+							<goal>copy-resources</goal>
+						</goals>
+						<configuration>
+							<outputDirectory>target/appassembler/</outputDirectory>
+							<resources>
+								<resource>
+									<directory>src/main/resources</directory>
+								</resource>
+							</resources>
+							<directoryMode>0755</directoryMode>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-antrun-plugin</artifactId>
+				<version>1.6</version>
+				<executions>
+					<execution>
+						<id>process-test-classes</id>
+						<phase>package</phase>
+						<configuration>
+							<target>
+								<chmod file="target/appassembler/bin/*)" perm="755" />
+							</target>
+						</configuration>
+						<goals>
+							<goal>run</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/getip.sh b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
new file mode 100755
index 0000000..e0cdf73
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
@@ -0,0 +1,21 @@
+#get the OS
+OS_NAME=`uname -a|awk '{print $1}'`
+LINUX_OS='Linux'
+
+if [ $OS_NAME = $LINUX_OS ];
+then
+        #Get IP Address
+        IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+	if [ "$IPADDR" = "" ]
+        then
+		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
+else
+        IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+	if [ "$IPADDR" = "" ]
+        then
+                IPADDR=`/sbin/ifconfig lo0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi
+
+fi
+echo $IPADDR
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh b/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
new file mode 100755
index 0000000..629bd90
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+   ssh $i "cd ${PREGELIX_PATH}; bin/startnc.sh"
+done
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh b/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
new file mode 100755
index 0000000..a0c2063
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
@@ -0,0 +1,3 @@
+bin/startcc.sh
+sleep 5
+bin/startAllNCs.sh
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
new file mode 100755
index 0000000..fe6cf27
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
@@ -0,0 +1,50 @@
+hostname
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+. conf/debugnc.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR2
+mkdir $NCTMP_DIR2
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR2
+mkdir $NCLOGS_DIR2
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS2 | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir
+	mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+#Get OS
+IPADDR=`bin/getip.sh`
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+NODEID=${NODEID}2
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS2
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR2
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
new file mode 100755
index 0000000..fe2551d
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
@@ -0,0 +1,25 @@
+#!/bin/bash
+hostname
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CCHOST=`bin/getip.sh`
+
+#Remove the temp dir
+rm -rf $CCTMP_DIR
+mkdir $CCTMP_DIR
+
+#Remove the logs dir
+rm -rf $CCLOGS_DIR
+mkdir $CCLOGS_DIR
+
+#Export JAVA_HOME and JAVA_OPTS
+export JAVA_HOME=$JAVA_HOME
+export JAVA_OPTS=$CCJAVA_OPTS
+
+#Launch hyracks cc script
+chmod -R 755 $HYRACKS_HOME
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -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 3 &> $CCLOGS_DIR/cc.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
new file mode 100755
index 0000000..6e0f90e
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
@@ -0,0 +1,49 @@
+hostname
+
+MY_NAME=`hostname`
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR
+mkdir $NCTMP_DIR
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR
+mkdir $NCLOGS_DIR
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir
+	mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+IPADDR=`bin/getip.sh`
+#echo $IPADDR
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
new file mode 100755
index 0000000..12367c1
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+   ssh $i "cd ${PREGELIX_PATH}; bin/stopnc.sh"
+done
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
new file mode 100755
index 0000000..4889934
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
@@ -0,0 +1,3 @@
+bin/stopAllNCs.sh
+sleep 2
+bin/stopcc.sh
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
new file mode 100755
index 0000000..c2f525a
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
@@ -0,0 +1,10 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep hyracks|awk '{print $2}'`
+echo $PID
+kill -9 $PID
+
+#Clean up CC temp dir
+rm -rf $CCTMP_DIR/*
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
new file mode 100755
index 0000000..03ce4e7
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
@@ -0,0 +1,23 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+
+if [ "$PID" == "" ]; then
+  USERID=`id | sed 's/^uid=//;s/(.*$//'`
+  PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+fi
+
+echo $PID
+kill -9 $PID
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir/*
+done
+
+#Clean up NC temp dir
+rm -rf $NCTMP_DIR/*
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties b/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
new file mode 100755
index 0000000..3b382f7
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
@@ -0,0 +1,37 @@
+#The CC port for Hyracks clients
+CC_CLIENTPORT=3099
+
+#The CC port for Hyracks cluster management
+CC_CLUSTERPORT=1099
+
+#The directory of hyracks binaries
+HYRACKS_HOME=../../../
+
+#The tmp directory for cc to install jars
+CCTMP_DIR=/tmp/t1
+
+#The tmp directory for nc to install jars
+NCTMP_DIR=/tmp/t2
+
+#The directory to put cc logs
+CCLOGS_DIR=$CCTMP_DIR/logs
+
+#The directory to put nc logs
+NCLOGS_DIR=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS="/tmp/t3,/tmp/t4"
+
+#The JAVA_HOME
+JAVA_HOME=$JAVA_HOME
+
+#The frame size of the internal dataflow engine
+FRAME_SIZE=65536
+
+#CC JAVA_OPTS
+CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+# Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties b/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
new file mode 100755
index 0000000..27afa26
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
@@ -0,0 +1,12 @@
+#The tmp directory for nc to install jars
+NCTMP_DIR2=/tmp/t-1
+
+#The directory to put nc logs
+NCLOGS_DIR2=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS2="/tmp/t-2,/tmp/t-3"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS2="-Xdebug -Xrunjdwp:transport=dt_socket,address=7003,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/master b/hyracks/hyracks-dist/src/main/resources/conf/master
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/master
@@ -0,0 +1 @@
+localhost
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/slaves b/hyracks/hyracks-dist/src/main/resources/conf/slaves
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/slaves
@@ -0,0 +1 @@
+localhost
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 82919ca..ba296ac 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -44,6 +44,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
index 2f0b00f..dd96db8 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
@@ -40,6 +40,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 61d4696..622942b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -52,30 +52,9 @@
 import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
 
 public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
-    private static class NoopNullWriterFactory implements INullWriterFactory {
-
-        private static final long serialVersionUID = 1L;
-        public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
-
-        private NoopNullWriterFactory() {
-        }
-
-        @Override
-        public INullWriter createNullWriter() {
-            return new INullWriter() {
-                @Override
-                public void writeNull(DataOutput out) throws HyracksDataException {
-                    try {
-                        out.writeShort(0);
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
-                    }
-                }
-            };
-        }
-    }
 
     /*
      * TPCH Customer table: CREATE TABLE CUSTOMER ( C_CUSTKEY INTEGER NOT NULL,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 1e60372..9233e39 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -15,9 +15,7 @@
 package edu.uci.ics.hyracks.tests.integration;
 
 import java.io.File;
-
 import org.junit.Test;
-
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -25,6 +23,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
@@ -45,6 +44,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
 
 public class TPCHCustomerOrderNestedLoopJoinTest extends AbstractIntegrationTest {
     private static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -165,7 +165,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4, false,
+                null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -239,7 +240,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, false,
+                null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -313,7 +315,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6, false,
+                null);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -334,4 +337,84 @@
         runTest(spec);
     }
 
+    @Test
+    public void customerOrderCIDOuterJoinMulti() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] custSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
+        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileSplit[] ordersSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
+        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
+
+        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
+
+        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
+        for (int j = 0; j < nullWriterFactories.length; j++) {
+            nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
+        }
+
+        NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, true,
+                nullWriterFactories);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(ordJoinConn, ordScanner, 0, join, 0);
+
+        IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(custJoinConn, custScanner, 0, join, 1);
+
+        IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(joinPrinterConn, join, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
new file mode 100644
index 0000000..d119509
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoopNullWriterFactory implements INullWriterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
+
+    private NoopNullWriterFactory() {
+    }
+
+    @Override
+    public INullWriter createNullWriter() {
+        return new INullWriter() {
+            @Override
+            public void writeNull(DataOutput out) throws HyracksDataException {
+                try {
+                    out.writeShort(0);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks/hyracks-examples/text-example/textclient/pom.xml
index d593ef9..901f1fb2 100644
--- a/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -40,6 +40,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
           <id>textclient</id>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 3170306..6b3f603 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -36,6 +36,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 01ccdef..0ad0ff0 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -199,7 +199,7 @@
 
         if ("nestedloop".equalsIgnoreCase(algo)) {
             join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize);
+                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize, false, null);
 
         } else if ("gracehash".equalsIgnoreCase(algo)) {
             join = new GraceHashJoinOperatorDescriptor(
diff --git a/hyracks/hyracks-hadoop-compat/pom.xml b/hyracks/hyracks-hadoop-compat/pom.xml
index 849efe3..3426293 100644
--- a/hyracks/hyracks-hadoop-compat/pom.xml
+++ b/hyracks/hyracks-hadoop-compat/pom.xml
@@ -25,6 +25,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-hdfs/pom.xml b/hyracks/hyracks-hdfs/pom.xml
new file mode 100644
index 0000000..b80dbd5
--- /dev/null
+++ b/hyracks/hyracks-hdfs/pom.xml
@@ -0,0 +1,118 @@
+<?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>hyracks</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>hyracks-hdfs</artifactId>
+	<name>hyracks-hdfs</name>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.6</source>
+					<target>1.6</target>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.7.2</version>
+				<configuration>
+					<forkMode>pertest</forkMode>
+					<includes>
+						<include>**/*TestSuite.java</include>
+						<include>**/*Test.java</include>
+					</includes>
+				</configuration>
+			</plugin>
+			<plugin>
+				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
+				<configuration>
+					<filesets>
+						<fileset>
+							<directory>.</directory>
+							<includes>
+								<include>edu*</include>
+								<include>actual*</include>
+								<include>build*</include>
+								<include>expect*</include>
+								<include>ClusterController*</include>
+								<include>edu.uci.*</include>
+							</includes>
+						</fileset>
+					</filesets>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+	<dependencies>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>3.8.1</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-api</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-dataflow-std</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-dataflow-common</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-core</artifactId>
+			<version>0.20.2</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-cc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-nc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-test</artifactId>
+			<version>0.20.2</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-io</artifactId>
+			<version>1.3.2</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+</project>
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
new file mode 100644
index 0000000..5923e1e
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ * 
+ * @param <K>
+ *            the key type
+ * @param <V>
+ *            the value type
+ */
+public interface IKeyValueParser<K, V> {
+
+    /**
+     * Parse a key-value pair returned by HDFS record reader to a tuple.
+     * when the parsers' internal buffer is full, it can flush the buffer to the writer
+     * 
+     * @param key
+     * @param value
+     * @param writer
+     * @throws HyracksDataException
+     */
+    public void parse(K key, V value, IFrameWriter writer) throws HyracksDataException;
+
+    /**
+     * Flush the residual tuples in the internal buffer to the writer.
+     * This method is called in the close() of HDFSReadOperatorDescriptor.
+     * 
+     * @param writer
+     * @throws HyracksDataException
+     */
+    public void flush(IFrameWriter writer) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
new file mode 100644
index 0000000..6e943ad
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ * 
+ * @param <K>
+ *            the key type
+ * @param <V>
+ *            the value type
+ */
+public interface IKeyValueParserFactory<K, V> extends Serializable {
+
+    /**
+     * This method creates a key-value parser.
+     * 
+     * @param ctx
+     *            the IHyracksTaskContext
+     * @return a key-value parser instance.
+     */
+    public IKeyValueParser<K, V> createKeyValueParser(IHyracksTaskContext ctx);
+
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
new file mode 100644
index 0000000..25b9523
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriter {
+
+    /**
+     * Write the tuple to the DataOutput.
+     * 
+     * @param output
+     *            the DataOutput channel
+     * @param tuple
+     *            the tuple to write
+     * @throws HyracksDataException
+     */
+    public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
new file mode 100644
index 0000000..839de8f
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.Serializable;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriterFactory extends Serializable {
+
+    /**
+     * @return a tuple writer instance
+     */
+    public ITupleWriter getTupleWriter();
+
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
new file mode 100644
index 0000000..4fa0164
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("deprecation")
+public class ConfFactory implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private byte[] confBytes;
+
+    public ConfFactory(JobConf conf) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            conf.write(dos);
+            confBytes = bos.toByteArray();
+            dos.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public JobConf getConf() throws HyracksDataException {
+        try {
+            JobConf conf = new JobConf();
+            DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+            conf.readFields(dis);
+            dis.close();
+            return conf;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..a0c821a3
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+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.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop old API.
+ * To use this operator, a user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final ConfFactory confFactory;
+    private final InputSplitsFactory splitsFactory;
+    private final String[] scheduledLocations;
+    private final IKeyValueParserFactory tupleParserFactory;
+    private final boolean[] executed;
+
+    /**
+     * The constructor of HDFSReadOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param rd
+     *            the output record descriptor
+     * @param conf
+     *            the Hadoop JobConf object, which contains the input format and the input paths
+     * @param splits
+     *            the array of FileSplits (HDFS chunks).
+     * @param scheduledLocations
+     *            the node controller names to scan the FileSplits, which is an one-to-one mapping. The String array
+     *            is obtained from the edu.cui.ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints(InputSplits[]).
+     * @param tupleParserFactory
+     *            the ITupleParserFactory implementation instance.
+     * @throws HyracksException
+     */
+    public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, JobConf conf, InputSplit[] splits,
+            String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+        super(spec, 0, 1);
+        try {
+            this.splitsFactory = new InputSplitsFactory(splits);
+            this.confFactory = new ConfFactory(conf);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+        this.scheduledLocations = scheduledLocations;
+        this.executed = new boolean[scheduledLocations.length];
+        Arrays.fill(executed, false);
+        this.tupleParserFactory = tupleParserFactory;
+        this.recordDescriptors[0] = rd;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final InputSplit[] inputSplits = splitsFactory.getSplits();
+        final JobConf conf = confFactory.getConf();
+
+        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+            private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void initialize() throws HyracksDataException {
+                try {
+                    IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+                    writer.open();
+                    InputFormat inputFormat = conf.getInputFormat();
+                    for (int i = 0; i < inputSplits.length; i++) {
+                        /**
+                         * read all the partitions scheduled to the current node
+                         */
+                        if (scheduledLocations[i].equals(nodeName)) {
+                            /**
+                             * pick an unread split to read
+                             * synchronize among simultaneous partitions in the same machine
+                             */
+                            synchronized (executed) {
+                                if (executed[i] == false) {
+                                    executed[i] = true;
+                                } else {
+                                    continue;
+                                }
+                            }
+
+                            /**
+                             * read the split
+                             */
+                            RecordReader reader = inputFormat.getRecordReader(inputSplits[i], conf, Reporter.NULL);
+                            Object key = reader.createKey();
+                            Object value = reader.createValue();
+                            while (reader.next(key, value) == true) {
+                                parser.parse(key, value, writer);
+                            }
+                        }
+                    }
+                    parser.flush(writer);
+                    writer.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..e29848c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop old API.
+ * To use this operator, a user need to provide an ITupleWriterFactory.
+ */
+@SuppressWarnings("deprecation")
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private ConfFactory confFactory;
+    private ITupleWriterFactory tupleWriterFactory;
+
+    /**
+     * The constructor of HDFSWriteOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param conf
+     *            the Hadoop JobConf which contains the output path
+     * @param tupleWriterFactory
+     *            the ITupleWriterFactory implementation object
+     * @throws HyracksException
+     */
+    public HDFSWriteOperatorDescriptor(JobSpecification spec, JobConf conf, ITupleWriterFactory tupleWriterFactory)
+            throws HyracksException {
+        super(spec, 1, 0);
+        this.confFactory = new ConfFactory(conf);
+        this.tupleWriterFactory = tupleWriterFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final JobConf conf = confFactory.getConf();
+        final String outputDirPath = FileOutputFormat.getOutputPath(conf).toString();
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+            private String fileName = outputDirPath + File.separator + "part-" + partition;
+            private FSDataOutputStream dos;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleReference tuple = new FrameTupleReference();
+            private ITupleWriter tupleWriter;
+
+            @Override
+            public void open() throws HyracksDataException {
+                tupleWriter = tupleWriterFactory.getTupleWriter();
+                try {
+                    FileSystem dfs = FileSystem.get(conf);
+                    dos = dfs.create(new Path(fileName), true);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                accessor.reset(buffer);
+                int tupleCount = accessor.getTupleCount();
+                for (int i = 0; i < tupleCount; i++) {
+                    tuple.reset(accessor, i);
+                    tupleWriter.write(dos, tuple);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    dos.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
new file mode 100644
index 0000000..9cc9ebc
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
@@ -0,0 +1,103 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class InputSplitsFactory implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private byte[] splitBytes;
+    private String splitClassName;
+
+    public InputSplitsFactory(InputSplit[] splits) throws HyracksDataException {
+        splitBytes = splitsToBytes(splits);
+        if (splits.length > 0) {
+            splitClassName = splits[0].getClass().getName();
+        }
+    }
+
+    public InputSplit[] getSplits() throws HyracksDataException {
+        return bytesToSplits(splitBytes);
+    }
+
+    /**
+     * Convert splits to bytes.
+     * 
+     * @param splits
+     *            input splits
+     * @return bytes which serialize the splits
+     * @throws IOException
+     */
+    private byte[] splitsToBytes(InputSplit[] splits) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            dos.writeInt(splits.length);
+            for (int i = 0; i < splits.length; i++) {
+                splits[i].write(dos);
+            }
+            dos.close();
+            return bos.toByteArray();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * Covert bytes to splits.
+     * 
+     * @param bytes
+     * @return
+     * @throws HyracksDataException
+     */
+    private InputSplit[] bytesToSplits(byte[] bytes) throws HyracksDataException {
+        try {
+            Class splitClass = Class.forName(splitClassName);
+            Constructor[] constructors = splitClass.getDeclaredConstructors();
+            Constructor defaultConstructor = null;
+            for (Constructor constructor : constructors) {
+                if (constructor.getParameterTypes().length == 0) {
+                    constructor.setAccessible(true);
+                    defaultConstructor = constructor;
+                }
+            }
+            ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+            DataInputStream dis = new DataInputStream(bis);
+            int size = dis.readInt();
+            InputSplit[] splits = new InputSplit[size];
+            for (int i = 0; i < size; i++) {
+                splits[i] = (InputSplit) defaultConstructor.newInstance();
+                splits[i].readFields(dis);
+            }
+            dis.close();
+            return splits;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..90c5977
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+    private RawBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int commonLength = Math.min(l1, l2);
+                for (int i = 0; i < commonLength; i++) {
+                    if (b1[s1 + i] != b2[s2 + i]) {
+                        return b1[s1 + i] - b2[s2 + i];
+                    }
+                }
+                int difference = l1 - l2;
+                return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..7895fec
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class RawBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static IBinaryHashFunctionFactory INSTANCE = new RawBinaryHashFunctionFactory();
+
+    private RawBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int value = 1;
+                int end = offset + length;
+                for (int i = offset; i < end; i++)
+                    value = value * 31 + (int) bytes[i];
+                return value;
+            }
+        };
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
new file mode 100644
index 0000000..c691f5d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+public class TextKeyValueParserFactory implements IKeyValueParserFactory<LongWritable, Text> {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx) {
+
+        final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+        final ByteBuffer buffer = ctx.allocateFrame();
+        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender.reset(buffer, true);
+
+        return new IKeyValueParser<LongWritable, Text>() {
+
+            @Override
+            public void parse(LongWritable key, Text value, IFrameWriter writer) throws HyracksDataException {
+                tb.reset();
+                tb.addField(value.getBytes(), 0, value.getLength());
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    FrameUtils.flushFrame(buffer, writer);
+                    appender.reset(buffer, true);
+                    if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                        throw new HyracksDataException("tuple cannot be appended into the frame");
+                    }
+                }
+            }
+
+            @Override
+            public void flush(IFrameWriter writer) throws HyracksDataException {
+                FrameUtils.flushFrame(buffer, writer);
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
new file mode 100644
index 0000000..d26721d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+public class TextTupleWriterFactory implements ITupleWriterFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ITupleWriter getTupleWriter() {
+        return new ITupleWriter() {
+            byte newLine = "\n".getBytes()[0];
+
+            @Override
+            public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException {
+                byte[] data = tuple.getFieldData(0);
+                int start = tuple.getFieldStart(0);
+                int len = tuple.getFieldLength(0);
+                try {
+                    output.write(data, start, len);
+                    output.writeByte(newLine);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
new file mode 100644
index 0000000..e7309d4
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
@@ -0,0 +1,210 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS.
+ * This class works for Hadoop old API.
+ */
+@SuppressWarnings("deprecation")
+public class Scheduler {
+
+    /** a list of NCs */
+    private String[] NCs;
+
+    /** a map from ip to NCs */
+    private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+
+    /** a map from the NC name to the index */
+    private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+
+    /**
+     * The constructor of the scheduler
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port) throws HyracksException {
+        try {
+            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            Map<String, NodeControllerInfo> ncNameToNcInfos = hcc.getNodeControllerInfos();
+            loadIPAddressToNCMap(ncNameToNcInfos);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        loadIPAddressToNCMap(ncNameToNcInfos);
+    }
+
+    /**
+     * Set location constraints for a file scan operator with a list of file splits
+     * 
+     * @throws HyracksDataException
+     */
+    public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
+        int[] capacity = new int[NCs.length];
+        Arrays.fill(capacity, 0);
+        String[] locations = new String[splits.length];
+        int slots = splits.length % capacity.length == 0 ? (splits.length / capacity.length) : (splits.length
+                / capacity.length + 1);
+
+        try {
+            Random random = new Random(System.currentTimeMillis());
+            boolean scheduled[] = new boolean[splits.length];
+            Arrays.fill(scheduled, false);
+
+            for (int i = 0; i < splits.length; i++) {
+                /**
+                 * get the location of all the splits
+                 */
+                String[] loc = splits[i].getLocations();
+                if (loc.length > 0) {
+                    for (int j = 0; j < loc.length; j++) {
+                        /**
+                         * get all the IP addresses from the name
+                         */
+                        InetAddress[] allIps = InetAddress.getAllByName(loc[j]);
+                        /**
+                         * iterate overa all ips
+                         */
+                        for (InetAddress ip : allIps) {
+                            /**
+                             * if the node controller exists
+                             */
+                            if (ipToNcMapping.get(ip.getHostAddress()) != null) {
+                                /**
+                                 * set the ncs
+                                 */
+                                List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
+                                int arrayPos = random.nextInt(dataLocations.size());
+                                String nc = dataLocations.get(arrayPos);
+                                int pos = ncNameToIndex.get(nc);
+                                /**
+                                 * check if the node is already full
+                                 */
+                                if (capacity[pos] < slots) {
+                                    locations[i] = nc;
+                                    capacity[pos]++;
+                                    scheduled[i] = true;
+                                }
+                            }
+                        }
+
+                        /**
+                         * break the loop for data-locations if the schedule has already been found
+                         */
+                        if (scheduled[i] == true) {
+                            break;
+                        }
+                    }
+                }
+            }
+
+            /**
+             * find the lowest index the current available NCs
+             */
+            int currentAvailableNC = 0;
+            for (int i = 0; i < capacity.length; i++) {
+                if (capacity[i] < slots) {
+                    currentAvailableNC = i;
+                    break;
+                }
+            }
+
+            /**
+             * schedule no-local file reads
+             */
+            for (int i = 0; i < splits.length; i++) {
+                // if there is no data-local NC choice, choose a random one
+                if (!scheduled[i]) {
+                    locations[i] = NCs[currentAvailableNC];
+                    capacity[currentAvailableNC]++;
+                    scheduled[i] = true;
+
+                    /**
+                     * move the available NC cursor to the next one
+                     */
+                    for (int j = currentAvailableNC; j < capacity.length; j++) {
+                        if (capacity[j] < slots) {
+                            currentAvailableNC = j;
+                            break;
+                        }
+                    }
+                }
+            }
+            return locations;
+        } catch (IOException e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * Load the IP-address-to-NC map from the NCNameToNCInfoMap
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        try {
+            NCs = new String[ncNameToNcInfos.size()];
+            int i = 0;
+
+            /**
+             * build the IP address to NC map
+             */
+            for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+                String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+                        .getHostAddress();
+                List<String> matchedNCs = ipToNcMapping.get(ipAddr);
+                if (matchedNCs == null) {
+                    matchedNCs = new ArrayList<String>();
+                    ipToNcMapping.put(ipAddr, matchedNCs);
+                }
+                matchedNCs.add(entry.getKey());
+                NCs[i] = entry.getKey();
+                i++;
+            }
+
+            /**
+             * set up the NC name to index mapping
+             */
+            for (i = 0; i < NCs.length; i++) {
+                ncNameToIndex.put(NCs[i], i);
+            }
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
new file mode 100644
index 0000000..d843d27
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapreduce.Job;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ConfFactory implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private byte[] confBytes;
+
+    public ConfFactory(Job conf) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            conf.getConfiguration().write(dos);
+            confBytes = bos.toByteArray();
+            dos.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public Job getConf() throws HyracksDataException {
+        try {
+            Job conf = new Job();
+            DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+            conf.getConfiguration().readFields(dis);
+            dis.close();
+            return conf;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
new file mode 100644
index 0000000..14dc70c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class FileSplitsFactory implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private byte[] splitBytes;
+    private String splitClassName;
+
+    public FileSplitsFactory(List<FileSplit> splits) throws HyracksDataException {
+        splitBytes = splitsToBytes(splits);
+        if (splits.size() > 0) {
+            splitClassName = splits.get(0).getClass().getName();
+        }
+    }
+
+    public List<FileSplit> getSplits() throws HyracksDataException {
+        return bytesToSplits(splitBytes);
+    }
+
+    /**
+     * Convert splits to bytes.
+     * 
+     * @param splits
+     *            input splits
+     * @return bytes which serialize the splits
+     * @throws IOException
+     */
+    private byte[] splitsToBytes(List<FileSplit> splits) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            dos.writeInt(splits.size());
+            int size = splits.size();
+            for (int i = 0; i < size; i++) {
+                splits.get(i).write(dos);
+            }
+            dos.close();
+            return bos.toByteArray();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * Covert bytes to splits.
+     * 
+     * @param bytes
+     * @return
+     * @throws HyracksDataException
+     */
+    private List<FileSplit> bytesToSplits(byte[] bytes) throws HyracksDataException {
+        try {
+            Class splitClass = Class.forName(splitClassName);
+            Constructor[] constructors = splitClass.getDeclaredConstructors();
+            Constructor defaultConstructor = null;
+            for (Constructor constructor : constructors) {
+                if (constructor.getParameterTypes().length == 0) {
+                    constructor.setAccessible(true);
+                    defaultConstructor = constructor;
+                }
+            }
+            ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+            DataInputStream dis = new DataInputStream(bis);
+            int size = dis.readInt();
+            List<FileSplit> splits = new ArrayList<FileSplit>();
+            for (int i = 0; i < size; i++) {
+                splits.add((FileSplit) defaultConstructor.newInstance());
+                splits.get(i).readFields(dis);
+            }
+            dis.close();
+            return splits;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..8a1ba6d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+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.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop new API.
+ * To use this operator, a user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings("rawtypes")
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final ConfFactory confFactory;
+    private final FileSplitsFactory splitsFactory;
+    private final String[] scheduledLocations;
+    private final IKeyValueParserFactory tupleParserFactory;
+    private final boolean[] executed;
+
+    /**
+     * The constructor of HDFSReadOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param rd
+     *            the output record descriptor
+     * @param conf
+     *            the Hadoop JobConf object, which contains the input format and the input paths
+     * @param splits
+     *            the array of FileSplits (HDFS chunks).
+     * @param scheduledLocations
+     *            the node controller names to scan the FileSplits, which is an one-to-one mapping. The String array
+     *            is obtained from the edu.cui.ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints(InputSplits[]).
+     * @param tupleParserFactory
+     *            the ITupleParserFactory implementation instance.
+     * @throws HyracksException
+     */
+    public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, Job conf, List<InputSplit> splits,
+            String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+        super(spec, 0, 1);
+        try {
+            List<FileSplit> fileSplits = new ArrayList<FileSplit>();
+            for (int i = 0; i < splits.size(); i++) {
+                fileSplits.add((FileSplit) splits.get(i));
+            }
+            this.splitsFactory = new FileSplitsFactory(fileSplits);
+            this.confFactory = new ConfFactory(conf);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+        this.scheduledLocations = scheduledLocations;
+        this.executed = new boolean[scheduledLocations.length];
+        Arrays.fill(executed, false);
+        this.tupleParserFactory = tupleParserFactory;
+        this.recordDescriptors[0] = rd;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final Job conf = confFactory.getConf();
+        final List<FileSplit> inputSplits = splitsFactory.getSplits();
+
+        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+            private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void initialize() throws HyracksDataException {
+                try {
+                    Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                    IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+                    writer.open();
+                    InputFormat inputFormat = ReflectionUtils.newInstance(conf.getInputFormatClass(),
+                            conf.getConfiguration());
+                    int size = inputSplits.size();
+                    for (int i = 0; i < size; i++) {
+                        /**
+                         * read all the partitions scheduled to the current node
+                         */
+                        if (scheduledLocations[i].equals(nodeName)) {
+                            /**
+                             * pick an unread split to read
+                             * synchronize among simultaneous partitions in the same machine
+                             */
+                            synchronized (executed) {
+                                if (executed[i] == false) {
+                                    executed[i] = true;
+                                } else {
+                                    continue;
+                                }
+                            }
+
+                            /**
+                             * read the split
+                             */
+                            TaskAttemptContext context = new TaskAttemptContext(conf.getConfiguration(),
+                                    new TaskAttemptID());
+                            RecordReader reader = inputFormat.createRecordReader(inputSplits.get(i), context);
+                            reader.initialize(inputSplits.get(i), context);
+                            while (reader.nextKeyValue() == true) {
+                                parser.parse(reader.getCurrentKey(), reader.getCurrentValue(), writer);
+                            }
+                        }
+                    }
+                    parser.flush(writer);
+                    writer.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..32bb9dc
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop new API.
+ * To use this operator, a user need to provide an ITupleWriterFactory.
+ */
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private ConfFactory confFactory;
+    private ITupleWriterFactory tupleWriterFactory;
+
+    /**
+     * The constructor of HDFSWriteOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param conf
+     *            the Hadoop JobConf which contains the output path
+     * @param tupleWriterFactory
+     *            the ITupleWriterFactory implementation object
+     * @throws HyracksException
+     */
+    public HDFSWriteOperatorDescriptor(JobSpecification spec, Job conf, ITupleWriterFactory tupleWriterFactory)
+            throws HyracksException {
+        super(spec, 1, 0);
+        this.confFactory = new ConfFactory(conf);
+        this.tupleWriterFactory = tupleWriterFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final Job conf = confFactory.getConf();
+        final String outputPath = FileOutputFormat.getOutputPath(new JobContext(conf.getConfiguration(), new JobID()))
+                .toString();
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+            private String fileName = outputPath + File.separator + "part-" + partition;
+            private FSDataOutputStream dos;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleReference tuple = new FrameTupleReference();
+            private ITupleWriter tupleWriter;
+
+            @Override
+            public void open() throws HyracksDataException {
+                tupleWriter = tupleWriterFactory.getTupleWriter();
+                try {
+                    FileSystem dfs = FileSystem.get(conf.getConfiguration());
+                    dos = dfs.create(new Path(fileName), true);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                accessor.reset(buffer);
+                int tupleCount = accessor.getTupleCount();
+                for (int i = 0; i < tupleCount; i++) {
+                    tuple.reset(accessor, i);
+                    tupleWriter.write(dos, tuple);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    dos.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
new file mode 100644
index 0000000..3445d68
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS.
+ * This class works for Hadoop new API.
+ */
+public class Scheduler {
+
+    /** a list of NCs */
+    private String[] NCs;
+
+    /** a map from ip to NCs */
+    private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+
+    /** a map from the NC name to the index */
+    private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+
+    /**
+     * The constructor of the scheduler
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port) throws HyracksException {
+        try {
+            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            Map<String, NodeControllerInfo> ncNameToNcInfos = hcc.getNodeControllerInfos();
+            loadIPAddressToNCMap(ncNameToNcInfos);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        loadIPAddressToNCMap(ncNameToNcInfos);
+    }
+
+    /**
+     * Set location constraints for a file scan operator with a list of file splits
+     * 
+     * @throws HyracksDataException
+     */
+    public String[] getLocationConstraints(List<InputSplit> splits) throws HyracksException {
+        int[] capacity = new int[NCs.length];
+        Arrays.fill(capacity, 0);
+        String[] locations = new String[splits.size()];
+        int slots = splits.size() % capacity.length == 0 ? (splits.size() / capacity.length) : (splits.size()
+                / capacity.length + 1);
+
+        try {
+            Random random = new Random(System.currentTimeMillis());
+            boolean scheduled[] = new boolean[splits.size()];
+            Arrays.fill(scheduled, false);
+
+            for (int i = 0; i < splits.size(); i++) {
+                /**
+                 * get the location of all the splits
+                 */
+                String[] loc = splits.get(i).getLocations();
+                if (loc.length > 0) {
+                    for (int j = 0; j < loc.length; j++) {
+                        /**
+                         * get all the IP addresses from the name
+                         */
+                        InetAddress[] allIps = InetAddress.getAllByName(loc[j]);
+                        /**
+                         * iterate overa all ips
+                         */
+                        for (InetAddress ip : allIps) {
+                            /**
+                             * if the node controller exists
+                             */
+                            if (ipToNcMapping.get(ip.getHostAddress()) != null) {
+                                /**
+                                 * set the ncs
+                                 */
+                                List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
+                                int arrayPos = random.nextInt(dataLocations.size());
+                                String nc = dataLocations.get(arrayPos);
+                                int pos = ncNameToIndex.get(nc);
+                                /**
+                                 * check if the node is already full
+                                 */
+                                if (capacity[pos] < slots) {
+                                    locations[i] = nc;
+                                    capacity[pos]++;
+                                    scheduled[i] = true;
+                                }
+                            }
+                        }
+
+                        /**
+                         * break the loop for data-locations if the schedule has already been found
+                         */
+                        if (scheduled[i] == true) {
+                            break;
+                        }
+                    }
+                }
+            }
+
+            /**
+             * find the lowest index the current available NCs
+             */
+            int currentAvailableNC = 0;
+            for (int i = 0; i < capacity.length; i++) {
+                if (capacity[i] < slots) {
+                    currentAvailableNC = i;
+                    break;
+                }
+            }
+
+            /**
+             * schedule no-local file reads
+             */
+            for (int i = 0; i < splits.size(); i++) {
+                // if there is no data-local NC choice, choose a random one
+                if (!scheduled[i]) {
+                    locations[i] = NCs[currentAvailableNC];
+                    capacity[currentAvailableNC]++;
+                    scheduled[i] = true;
+
+                    /**
+                     * move the available NC cursor to the next one
+                     */
+                    for (int j = currentAvailableNC; j < capacity.length; j++) {
+                        if (capacity[j] < slots) {
+                            currentAvailableNC = j;
+                            break;
+                        }
+                    }
+                }
+            }
+            return locations;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * Load the IP-address-to-NC map from the NCNameToNCInfoMap
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        try {
+            NCs = new String[ncNameToNcInfos.size()];
+            int i = 0;
+
+            /**
+             * build the IP address to NC map
+             */
+            for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+                String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+                        .getHostAddress();
+                List<String> matchedNCs = ipToNcMapping.get(ipAddr);
+                if (matchedNCs == null) {
+                    matchedNCs = new ArrayList<String>();
+                    ipToNcMapping.put(ipAddr, matchedNCs);
+                }
+                matchedNCs.add(entry.getKey());
+                NCs[i] = entry.getKey();
+                i++;
+            }
+
+            /**
+             * set up the NC name to index mapping
+             */
+            for (i = 0; i < NCs.length; i++) {
+                ncNameToIndex.put(NCs[i], i);
+            }
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
new file mode 100644
index 0000000..2686077
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs.dataflow package,
+ * the operators for the Hadoop old API.
+ */
+@SuppressWarnings({ "deprecation" })
+public class DataflowTest extends TestCase {
+
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+    private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+    private static final String HDFS_INPUT_PATH = "/customer/";
+    private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+    private static final String HYRACKS_APP_NAME = "DataflowTest";
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 2;
+
+    @Override
+    public void setUp() throws Exception {
+        cleanupStores();
+        HyracksUtils.init();
+        HyracksUtils.createApp(HYRACKS_APP_NAME);
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * Start the HDFS cluster and setup the data files
+     * 
+     * @throws IOException
+     */
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_INPUT_PATH);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        dfs.mkdirs(dest);
+        dfs.mkdirs(result);
+        dfs.copyFromLocalFile(src, dest);
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    /**
+     * Test a job with only HDFS read and writes.
+     * 
+     * @throws Exception
+     */
+    public void testHDFSReadWriteOperators() throws Exception {
+        FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+        FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+        conf.setInputFormat(TextInputFormat.class);
+
+        Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        InputSplit[] splits = conf.getInputFormat().getSplits(conf, numberOfNC * 4);
+
+        String[] readSchedule = scheduler.getLocationConstraints(splits);
+        JobSpecification jobSpec = new JobSpecification();
+        RecordDescriptor recordDesc = new RecordDescriptor(
+                new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+        String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+                HyracksUtils.NC2_ID };
+        HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+                readSchedule, new TextKeyValueParserFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+        ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+                new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+        HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+                new TextTupleWriterFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+        jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+        jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+                0, writeOperator, 0);
+        jobSpec.addRoot(writeOperator);
+
+        IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+                HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        client.waitForCompletion(jobId);
+
+        Assert.assertEquals(true, checkResults());
+    }
+
+    /**
+     * Check if the results are correct
+     * 
+     * @return true if correct
+     * @throws Exception
+     */
+    private boolean checkResults() throws Exception {
+        FileSystem dfs = FileSystem.get(conf);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        Path actual = new Path(ACTUAL_RESULT_DIR);
+        dfs.copyToLocalFile(result, actual);
+
+        TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+                ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+        return true;
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+        HyracksUtils.deinit();
+        cleanupHDFS();
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..4b8a278
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -0,0 +1,210 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+@SuppressWarnings("deprecation")
+public class SchedulerTest extends TestCase {
+
+    /**
+     * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSimple() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        InputSplit[] fileSplits = new InputSplit[6];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc5", "nc6" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerLargerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        InputSplit[] fileSplits = new InputSplit[12];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" });
+        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
+                "nc6", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        InputSplit[] fileSplits = new InputSplit[12];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
+                "nc5", "nc6" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFSOdd() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        InputSplit[] fileSplits = new InputSplit[13];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[12] = new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" });
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc3", "nc4", "nc2",
+                "nc4", "nc5", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
new file mode 100644
index 0000000..d44b75a
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.utils;
+
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class HyracksUtils {
+
+    public static final String NC1_ID = "nc1";
+    public static final String NC2_ID = "nc2";
+
+    public static final int DEFAULT_HYRACKS_CC_PORT = 1099;
+    public static final int TEST_HYRACKS_CC_PORT = 1099;
+    public static final int TEST_HYRACKS_CC_CLIENT_PORT = 2099;
+    public static final String CC_HOST = "localhost";
+
+    public static final int FRAME_SIZE = 65536;
+
+    private static ClusterControllerService cc;
+    private static NodeControllerService nc1;
+    private static NodeControllerService nc2;
+    private static IHyracksClientConnection hcc;
+
+    public static void init() throws Exception {
+        CCConfig ccConfig = new CCConfig();
+        ccConfig.clientNetIpAddress = CC_HOST;
+        ccConfig.clusterNetIpAddress = CC_HOST;
+        ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
+        ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
+        ccConfig.defaultMaxJobAttempts = 0;
+        ccConfig.jobHistorySize = 0;
+        ccConfig.profileDumpPeriod = -1;
+
+        // cluster controller
+        cc = new ClusterControllerService(ccConfig);
+        cc.start();
+
+        // two node controllers
+        NCConfig ncConfig1 = new NCConfig();
+        ncConfig1.ccHost = "localhost";
+        ncConfig1.clusterNetIPAddress = "localhost";
+        ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
+        ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.nodeId = NC1_ID;
+        nc1 = new NodeControllerService(ncConfig1);
+        nc1.start();
+
+        NCConfig ncConfig2 = new NCConfig();
+        ncConfig2.ccHost = "localhost";
+        ncConfig2.clusterNetIPAddress = "localhost";
+        ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
+        ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.nodeId = NC2_ID;
+        nc2 = new NodeControllerService(ncConfig2);
+        nc2.start();
+
+        // hyracks connection
+        hcc = new HyracksConnection(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
+    }
+
+    public static void destroyApp(String hyracksAppName) throws Exception {
+        hcc.destroyApplication(hyracksAppName);
+    }
+
+    public static void createApp(String hyracksAppName) throws Exception {
+        hcc.createApplication(hyracksAppName, null);
+    }
+
+    public static void deinit() throws Exception {
+        nc2.stop();
+        nc1.stop();
+        cc.stop();
+    }
+
+    public static void runJob(JobSpecification spec, String appName) throws Exception {
+        spec.setFrameSize(FRAME_SIZE);
+        JobId jobId = hcc.startJob(appName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        hcc.waitForCompletion(jobId);
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
new file mode 100644
index 0000000..3826688
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.utils;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+
+public class TestUtils {
+
+    public static void compareWithResult(File expectedFile, File actualFile) throws Exception {
+        BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
+        BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
+        String lineExpected, lineActual;
+        int num = 1;
+        try {
+            while ((lineExpected = readerExpected.readLine()) != null) {
+                lineActual = readerActual.readLine();
+                // Assert.assertEquals(lineExpected, lineActual);
+                if (lineActual == null) {
+                    throw new Exception("Actual result changed at line " + num + ":\n< " + lineExpected + "\n> ");
+                }
+                if (!equalStrings(lineExpected, lineActual)) {
+                    throw new Exception("Result for changed at line " + num + ":\n< " + lineExpected + "\n> "
+                            + lineActual);
+                }
+                ++num;
+            }
+            lineActual = readerActual.readLine();
+            if (lineActual != null) {
+                throw new Exception("Actual result changed at line " + num + ":\n< \n> " + lineActual);
+            }
+        } finally {
+            readerExpected.close();
+            readerActual.close();
+        }
+    }
+
+    private static boolean equalStrings(String s1, String s2) {
+        String[] rowsOne = s1.split("\n");
+        String[] rowsTwo = s2.split("\n");
+
+        if (rowsOne.length != rowsTwo.length)
+            return false;
+
+        for (int i = 0; i < rowsOne.length; i++) {
+            String row1 = rowsOne[i];
+            String row2 = rowsTwo[i];
+
+            if (row1.equals(row2))
+                continue;
+
+            String[] fields1 = row1.split(",");
+            String[] fields2 = row2.split(",");
+
+            for (int j = 0; j < fields1.length; j++) {
+                if (fields1[j].equals(fields2[j])) {
+                    continue;
+                } else if (fields1[j].indexOf('.') < 0) {
+                    return false;
+                } else {
+                    fields1[j] = fields1[j].split("=")[1];
+                    fields2[j] = fields2[j].split("=")[1];
+                    Double double1 = Double.parseDouble(fields1[j]);
+                    Double double2 = Double.parseDouble(fields2[j]);
+                    float float1 = (float) double1.doubleValue();
+                    float float2 = (float) double2.doubleValue();
+
+                    if (Math.abs(float1 - float2) == 0)
+                        continue;
+                    else {
+                        return false;
+                    }
+                }
+            }
+        }
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
new file mode 100644
index 0000000..508ba07
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+import edu.uci.ics.hyracks.hdfs2.scheduler.Scheduler;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs2.dataflow package,
+ * the operators for the Hadoop new API.
+ */
+public class DataflowTest extends TestCase {
+
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+    private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+    private static final String HDFS_INPUT_PATH = "/customer/";
+    private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+    private static final String HYRACKS_APP_NAME = "DataflowTest";
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private Job conf;
+    private int numberOfNC = 2;
+
+    @Override
+    public void setUp() throws Exception {
+        conf = new Job();
+        cleanupStores();
+        HyracksUtils.init();
+        HyracksUtils.createApp(HYRACKS_APP_NAME);
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * Start the HDFS cluster and setup the data files
+     * 
+     * @throws IOException
+     */
+    private void startHDFS() throws IOException {
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf.getConfiguration(), numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf.getConfiguration());
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_INPUT_PATH);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        dfs.mkdirs(dest);
+        dfs.mkdirs(result);
+        dfs.copyFromLocalFile(src, dest);
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.getConfiguration().writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    /**
+     * Test a job with only HDFS read and writes.
+     * 
+     * @throws Exception
+     */
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    public void testHDFSReadWriteOperators() throws Exception {
+        FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+        FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+        conf.setInputFormatClass(TextInputFormat.class);
+
+        Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        InputFormat inputFormat = ReflectionUtils.newInstance(conf.getInputFormatClass(), conf.getConfiguration());
+        List<InputSplit> splits = inputFormat.getSplits(new JobContext(conf.getConfiguration(), new JobID()));
+
+        String[] readSchedule = scheduler.getLocationConstraints(splits);
+        JobSpecification jobSpec = new JobSpecification();
+        RecordDescriptor recordDesc = new RecordDescriptor(
+                new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+        String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+                HyracksUtils.NC2_ID };
+        HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+                readSchedule, new TextKeyValueParserFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+        ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+                new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+        HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+                new TextTupleWriterFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+        jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+        jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+                0, writeOperator, 0);
+        jobSpec.addRoot(writeOperator);
+
+        IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+                HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        client.waitForCompletion(jobId);
+
+        Assert.assertEquals(true, checkResults());
+    }
+
+    /**
+     * Check if the results are correct
+     * 
+     * @return true if correct
+     * @throws Exception
+     */
+    private boolean checkResults() throws Exception {
+        FileSystem dfs = FileSystem.get(conf.getConfiguration());
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        Path actual = new Path(ACTUAL_RESULT_DIR);
+        dfs.copyToLocalFile(result, actual);
+
+        TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+                ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+        return true;
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+        HyracksUtils.deinit();
+        cleanupHDFS();
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..ea2af13
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+/**
+ * Test case for the new HDFS API scheduler
+ * 
+ */
+public class SchedulerTest extends TestCase {
+
+    /**
+     * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSimple() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc5", "nc6" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerLargerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
+                "nc6", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
+                "nc5", "nc6" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFSOdd() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc3", "nc4", "nc2",
+                "nc4", "nc5", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/src/test/resources/data/customer.tbl b/hyracks/hyracks-hdfs/src/test/resources/data/customer.tbl
new file mode 100644
index 0000000..5d39c80
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/data/customer.tbl
@@ -0,0 +1,150 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
diff --git a/hyracks/hyracks-hdfs/src/test/resources/expected/part-0 b/hyracks/hyracks-hdfs/src/test/resources/expected/part-0
new file mode 100755
index 0000000..ce3b00c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/expected/part-0
@@ -0,0 +1,150 @@
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
diff --git a/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/core-site.xml b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/core-site.xml
new file mode 100644
index 0000000..47dfac5
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/core-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+    <name>fs.default.name</name>
+    <value>hdfs://127.0.0.1:31888</value>
+</property>
+<property>
+    <name>hadoop.tmp.dir</name>
+    <value>/tmp/hadoop</value>
+</property>
+
+
+</configuration>
diff --git a/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/hdfs-site.xml b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/hdfs-site.xml
new file mode 100644
index 0000000..8d29b1d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/hdfs-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+   <name>dfs.replication</name>
+   <value>1</value>
+</property>
+
+<property>
+	<name>dfs.block.size</name>
+	<value>65536</value>
+</property>
+
+</configuration>
diff --git a/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/log4j.properties b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/log4j.properties
new file mode 100755
index 0000000..d5e6004
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/log4j.properties
@@ -0,0 +1,94 @@
+# Define some default values that can be overridden by system properties
+hadoop.root.logger=FATAL,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshhold=FATAL
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this 
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+#
+# TaskLog Appender
+#
+
+#Default values
+hadoop.tasklog.taskid=null
+hadoop.tasklog.noKeepSplits=4
+hadoop.tasklog.totalLogFileSize=100
+hadoop.tasklog.purgeLogSplits=true
+hadoop.tasklog.logsRetainHours=12
+
+log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
+log4j.appender.TLA.taskId=${hadoop.tasklog.taskid}
+log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize}
+
+log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+
+#
+# Rolling File Appender
+#
+
+#log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+#log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Logfile size and and 30-day backups
+#log4j.appender.RFA.MaxFileSize=1MB
+#log4j.appender.RFA.MaxBackupIndex=30
+
+#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+#
+# FSNamesystem Audit logging
+# All audit events are logged at INFO level
+#
+log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=WARN
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG
+#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+
+# Jets3t library
+log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.metrics.jvm.EventCounter
diff --git a/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/mapred-site.xml b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/mapred-site.xml
new file mode 100644
index 0000000..39b6505
--- /dev/null
+++ b/hyracks/hyracks-hdfs/src/test/resources/hadoop/conf/mapred-site.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+  <property>
+    <name>mapred.job.tracker</name>
+    <value>localhost:29007</value>
+  </property>
+  <property>
+     <name>mapred.tasktracker.map.tasks.maximum</name>
+     <value>20</value>
+  </property>
+   <property>
+      <name>mapred.tasktracker.reduce.tasks.maximum</name>
+      <value>20</value>
+   </property>
+   <property>
+      <name>mapred.max.split.size</name>
+      <value>2048</value>
+   </property>
+
+</configuration>
diff --git a/hyracks/hyracks-server/pom.xml b/hyracks/hyracks-server/pom.xml
index 2699c27..6c6640e 100644
--- a/hyracks/hyracks-server/pom.xml
+++ b/hyracks/hyracks-server/pom.xml
@@ -22,6 +22,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index d00bea6..6744f70 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -15,13 +15,13 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -41,7 +41,7 @@
 
     private FrameTupleAccessor accessor;
     private ArrayTupleBuilder builder;
-    private DataOutput builderDos;
+    private GrowableArray builderFieldData;
     private FrameTupleAppender appender;
     private ByteBuffer writeBuffer;
 
@@ -60,7 +60,7 @@
         accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
         writeBuffer = ctx.allocateFrame();
         builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
-        builderDos = builder.getDataOutput();
+        builderFieldData = builder.getFieldData();
         appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
         writer.open();
@@ -87,7 +87,7 @@
                     builder.reset();
                     try {
                         IToken token = tokenizer.getToken();
-                        token.serializeToken(builderDos);
+                        token.serializeToken(builderFieldData);
                         builder.addFieldEndOffset();
                     } catch (IOException e) {
                         throw new HyracksDataException(e.getMessage());
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 99e76dc..3525fc3 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -202,7 +202,9 @@
     public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
         // Create entry in btree for last inverted list.
         InvertedIndexBulkLoadContext ctx = (InvertedIndexBulkLoadContext) ictx;
-        createAndInsertBTreeTuple(ctx);
+        if (ctx.lastTuple.getFieldData(0) != null) {
+            createAndInsertBTreeTuple(ctx);
+        }
         btree.endBulkLoad(ctx.btreeBulkLoadCtx);
         ctx.deinit();
     }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 1f886a2..af5dad3 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -28,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -75,7 +75,7 @@
     protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
             new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
     protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
-    protected DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
+    protected GrowableArray queryTokenFieldData = queryTokenBuilder.getFieldData();
     protected FrameTupleAppender queryTokenAppender;
     protected ByteBuffer queryTokenFrame;
 
@@ -158,7 +158,7 @@
             queryTokenBuilder.reset();
             try {
                 IToken token = queryTokenizer.getToken();
-                token.serializeToken(queryTokenDos);
+                token.serializeToken(queryTokenFieldData);
                 queryTokenBuilder.addFieldEndOffset();
                 // WARNING: assuming one frame is big enough to hold all tokens
                 queryTokenAppender.append(queryTokenBuilder.getFieldEndOffsets(), queryTokenBuilder.getByteArray(), 0,
@@ -294,7 +294,8 @@
         boolean advanceCursor = true;
         boolean advancePrevResult = false;
         int resultTidx = 0;
-
+        currentNumResults = 0;
+        
         resultFrameTupleAcc.reset(prevCurrentBuffer);
         resultFrameTupleApp.reset(newCurrentBuffer, true);
 
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
index 65afa65..2f60952 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public abstract class AbstractUTF8Token implements IToken {
     public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
@@ -97,8 +98,8 @@
     }
 
     @Override
-    public void serializeTokenCount(DataOutput dos) throws IOException {
-        handleCountTypeTag(dos);
-        dos.writeInt(tokenCount);
+    public void serializeTokenCount(GrowableArray out) throws IOException {
+        handleCountTypeTag(out.getDataOutput());
+        out.getDataOutput().writeInt(tokenCount);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
index b7bb828..a1a4354 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public class HashedUTF8NGramToken extends UTF8NGramToken {
     public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
@@ -30,8 +30,8 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
 
         int hash = GOLDEN_RATIO_32;
 
@@ -59,6 +59,6 @@
         // token count
         hash += tokenCount;
 
-        dos.writeInt(hash);
+        out.getDataOutput().writeInt(hash);
     }
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
index 42ed053..20405c6 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public class HashedUTF8WordToken extends UTF8WordToken {
 
@@ -76,12 +76,12 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
+    public void serializeToken(GrowableArray out) throws IOException {
         if (tokenTypeTag > 0) {
-            dos.write(tokenTypeTag);
+            out.getDataOutput().write(tokenTypeTag);
         }
 
         // serialize hash value
-        dos.writeInt(hash);
+        out.getDataOutput().writeInt(hash);
     }
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
index c1840d7..47467a1 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
@@ -19,9 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
 public interface IToken {
 	public byte[] getData();
 
@@ -34,7 +35,7 @@
 	public void reset(byte[] data, int start, int length, int tokenLength,
 			int tokenCount);
 
-	public void serializeToken(DataOutput dos) throws IOException;
+	public void serializeToken(GrowableArray out) throws IOException;
 
-	public void serializeTokenCount(DataOutput dos) throws IOException;
+	public void serializeTokenCount(GrowableArray out) throws IOException;
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
index 59cadc8..8cb9818 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
@@ -49,34 +49,39 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
 
         // regular chars
         int numRegChars = tokenLength - numPreChars - numPostChars;
 
         // assuming pre and post char need 1-byte each in utf8
-        int tokenUTF8Len = getLowerCaseUTF8Len(numRegChars) + numPreChars + numPostChars;
+        int tokenUTF8Len = numPreChars + numPostChars;
 
-        // write utf8 length indicator
-        StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
 
         // pre chars
         for (int i = 0; i < numPreChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(PRECHAR, dos);
+            StringUtils.writeCharAsModifiedUTF8(PRECHAR, out.getDataOutput());
         }
 
         int pos = start;
         for (int i = 0; i < numRegChars; i++) {
             char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            StringUtils.writeCharAsModifiedUTF8(c, dos);
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
             pos += UTF8StringPointable.charSize(data, pos);
         }
 
         // post chars
         for (int i = 0; i < numPostChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, dos);
+            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, out.getDataOutput());
         }
+
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
     }
 
     public void setNumPrePostChars(int numPreChars, int numPostChars) {
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
index 97a1e12..9d7fe7c 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8WordToken extends AbstractUTF8Token {
@@ -32,16 +32,20 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
-
-        int tokenUTF8Len = getLowerCaseUTF8Len(tokenLength);
-        StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
+        int tokenUTF8Len = 0;
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
         int pos = start;
         for (int i = 0; i < tokenLength; i++) {
             char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            StringUtils.writeCharAsModifiedUTF8(c, dos);
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
             pos += UTF8StringPointable.charSize(data, pos);
         }
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
     }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 9818dce..d9b7b97 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -40,9 +40,10 @@
             IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
-            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
+            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
+            IOperationCallbackProvider opCallbackProvider) {
         super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+                comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
         this.keyFields = keyFields;
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
index 5f15a91..3fb6407 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
@@ -33,6 +33,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
@@ -41,207 +42,196 @@
 
 public class NGramTokenizerTest {
 
-	private char PRECHAR = '#';
-	private char POSTCHAR = '$';
+    private char PRECHAR = '#';
+    private char POSTCHAR = '$';
 
-	private String str = "Jürgen S. Generic's Car";
-	private byte[] inputBuffer;
+    private String str = "Jürgen S. Generic's Car";
+    private byte[] inputBuffer;
 
-	private int gramLength = 3;
+    private int gramLength = 3;
 
-	private void getExpectedGrams(String s, int gramLength,
-			ArrayList<String> grams, boolean prePost) {
+    private void getExpectedGrams(String s, int gramLength, ArrayList<String> grams, boolean prePost) {
 
-		String tmp = s.toLowerCase();
-		if (prePost) {
-			StringBuilder preBuilder = new StringBuilder();
-			for (int i = 0; i < gramLength - 1; i++) {
-				preBuilder.append(PRECHAR);
-			}
-			String pre = preBuilder.toString();
+        String tmp = s.toLowerCase();
+        if (prePost) {
+            StringBuilder preBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                preBuilder.append(PRECHAR);
+            }
+            String pre = preBuilder.toString();
 
-			StringBuilder postBuilder = new StringBuilder();
-			for (int i = 0; i < gramLength - 1; i++) {
-				postBuilder.append(POSTCHAR);
-			}
-			String post = postBuilder.toString();
+            StringBuilder postBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                postBuilder.append(POSTCHAR);
+            }
+            String post = postBuilder.toString();
 
-			tmp = pre + s.toLowerCase() + post;
-		}
+            tmp = pre + s.toLowerCase() + post;
+        }
 
-		for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
-			String gram = tmp.substring(i, i + gramLength);
-			grams.add(gram);
-		}
-	}
+        for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
+            String gram = tmp.substring(i, i + gramLength);
+            grams.add(gram);
+        }
+    }
 
-	@Before
-	public void init() throws Exception {
-		// serialize string into bytes
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		DataOutput dos = new DataOutputStream(baos);
-		dos.writeUTF(str);
-		inputBuffer = baos.toByteArray();
-	}
+    @Before
+    public void init() throws Exception {
+        // serialize string into bytes
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dos = new DataOutputStream(baos);
+        dos.writeUTF(str);
+        inputBuffer = baos.toByteArray();
+    }
 
-	void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost)
-			throws IOException {
-		HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, false, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, false,
+                false, tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
-		ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-		HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
-		for (String s : expectedGrams) {
-			Integer count = gramCounts.get(s);
-			if (count == null) {
-				count = 1;
-				gramCounts.put(s, count);
-			} else {
-				count++;
-			}
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
+        for (String s : expectedGrams) {
+            Integer count = gramCounts.get(s);
+            if (count == null) {
+                count = 1;
+                gramCounts.put(s, count);
+            } else {
+                count++;
+            }
 
-			int hash = tokenHash(s, count);
-			expectedHashedGrams.add(hash);
-		}
+            int hash = tokenHash(s, count);
+            expectedHashedGrams.add(hash);
+        }
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			Integer hashedGram = in.readInt();
+            Integer hashedGram = in.readInt();
 
-			// System.out.println(hashedGram);
+            // System.out.println(hashedGram);
 
-			Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost)
-			throws IOException {
-		HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, true, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
-		ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-		for (String s : expectedGrams) {
-			int hash = tokenHash(s, 1);
-			expectedHashedGrams.add(hash);
-		}
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        for (String s : expectedGrams) {
+            int hash = tokenHash(s, 1);
+            expectedHashedGrams.add(hash);
+        }
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			Integer hashedGram = in.readInt();
+            Integer hashedGram = in.readInt();
 
-			// System.out.println(hashedGram);
+            // System.out.println(hashedGram);
 
-			Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	void runTestNGramTokenizerWithUTF8Tokens(boolean prePost)
-			throws IOException {
-		UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, true, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithUTF8Tokens(boolean prePost) throws IOException {
+        UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			String strGram = in.readUTF();
+            String strGram = in.readUTF();
 
-			// System.out.println("\"" + strGram + "\"");
+            // System.out.println("\"" + strGram + "\"");
 
-			Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
+            Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	@Test
-	public void testNGramTokenizerWithCountedHashedUTF8Tokens()
-			throws Exception {
-		runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
-		runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithCountedHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
+    }
 
-	@Test
-	public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
-		runTestNGramTokenizerWithHashedUTF8Tokens(false);
-		runTestNGramTokenizerWithHashedUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithHashedUTF8Tokens(true);
+    }
 
-	@Test
-	public void testNGramTokenizerWithUTF8Tokens() throws IOException {
-		runTestNGramTokenizerWithUTF8Tokens(false);
-		runTestNGramTokenizerWithUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithUTF8Tokens() throws IOException {
+        runTestNGramTokenizerWithUTF8Tokens(false);
+        runTestNGramTokenizerWithUTF8Tokens(true);
+    }
 
-	public int tokenHash(String token, int tokenCount) {
-		int h = AbstractUTF8Token.GOLDEN_RATIO_32;
-		for (int i = 0; i < token.length(); i++) {
-			h ^= token.charAt(i);
-			h *= AbstractUTF8Token.GOLDEN_RATIO_32;
-		}
-		return h + tokenCount;
-	}
+    public int tokenHash(String token, int tokenCount) {
+        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
+        for (int i = 0; i < token.length(); i++) {
+            h ^= token.charAt(i);
+            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
+        }
+        return h + tokenCount;
+    }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index c3c9b99..47a068b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -27,6 +27,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -112,20 +113,19 @@
 	}
 
 	private class TokenIdPair implements Comparable<TokenIdPair> {
-		public ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-		public DataOutputStream dos = new DataOutputStream(baaos);
+	    public final GrowableArray tokenStorage = new GrowableArray();
 		public int id;
 
 		TokenIdPair(IToken token, int id) throws IOException {
-			token.serializeToken(dos);
+			token.serializeToken(tokenStorage);
 			this.id = id;
 		}
 
 		@Override
 		public int compareTo(TokenIdPair o) {
-			int cmp = btreeBinCmps[0].compare(baaos.getByteArray(), 0,
-					baaos.getByteArray().length, o.baaos.getByteArray(), 0,
-					o.baaos.getByteArray().length);
+			int cmp = btreeBinCmps[0].compare(tokenStorage.getByteArray(), 0,
+			        tokenStorage.getByteArray().length, o.tokenStorage.getByteArray(), 0,
+					o.tokenStorage.getByteArray().length);
 			if (cmp == 0) {
 				return id - o.id;
 			} else {
@@ -157,8 +157,8 @@
 
 		for (TokenIdPair t : pairs) {
 			tb.reset();
-			tb.addField(t.baaos.getByteArray(), 0,
-					t.baaos.getByteArray().length);
+			tb.addField(t.tokenStorage.getByteArray(), 0,
+					t.tokenStorage.getByteArray().length);
 			IntegerSerializerDeserializer.INSTANCE.serialize(t.id, tb.getDataOutput());
 			tb.addFieldEndOffset();
 			tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
index 53fb96d..810c5f5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
@@ -34,6 +34,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
@@ -127,14 +128,13 @@
             tokenizer.next();
 
             // serialize token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             Integer hashedToken = in.readInt();
@@ -159,14 +159,13 @@
             tokenizer.next();
 
             // serialize token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             Integer hashedToken = in.readInt();
@@ -191,14 +190,13 @@
             tokenizer.next();
 
             // serialize hashed token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             String strToken = in.readUTF();
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 7f2fccb..b699542 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -1,4 +1,4 @@
-
+<?xml version="1.0" encoding="UTF-8"?>
 <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>
@@ -8,24 +8,9 @@
   <name>hyracks</name>
 
   <properties>
-    <jvm.extraargs />
+    <jvm.extraargs/>
   </properties>
 
-  <profiles>
-    <profile>
-      <id>macosx</id>
-      <activation>
-        <os>
-          <name>mac os x</name>
-        </os>
-        <jdk>1.7</jdk>
-      </activation>
-      <properties>
-        <jvm.extraargs>-Djava.nio.channels.spi.SelectorProvider=sun.nio.ch.KQueueSelectorProvider</jvm.extraargs>
-      </properties>
-    </profile>
-  </profiles>
-
   <build>
     <plugins>
       <plugin>
@@ -44,6 +29,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.13</version>
         <configuration>
             <forkMode>pertest</forkMode>
             <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -57,6 +43,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-changelog-plugin</artifactId>
+        <version>2.2</version>
       </plugin>
     </plugins>
   </reporting>
@@ -118,5 +105,7 @@
     <module>hyracks-hadoop-compat</module>
     <!--module>hyracks-yarn</module-->
     <module>hyracks-maven-plugins</module>
+    <module>hyracks-hdfs</module>
+    <module>hyracks-dist</module>
   </modules>
 </project>
diff --git a/pregelix/pom.xml b/pregelix/pom.xml
index 4b6a968..7d08fb7 100644
--- a/pregelix/pom.xml
+++ b/pregelix/pom.xml
@@ -44,6 +44,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.13</version>
         <configuration>
             <forkMode>pertest</forkMode>
             <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -57,6 +58,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-changelog-plugin</artifactId>
+        <version>2.2</version>
       </plugin>
     </plugins>
   </reporting>
diff --git a/pregelix/pregelix-api/pom.xml b/pregelix/pregelix-api/pom.xml
index 66a0186..2caa93b 100644
--- a/pregelix/pregelix-api/pom.xml
+++ b/pregelix/pregelix-api/pom.xml
@@ -41,6 +41,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
index 4af35fe..e5f42fe 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Edge.java
@@ -42,7 +42,7 @@
     private E edgeValue = null;
     /** Configuration - Used to instantiate classes */
     private Configuration conf = null;
-    /** Whether the edgeValue field is not null*/
+    /** Whether the edgeValue field is not null */
     private boolean hasEdgeValue = false;
 
     /**
@@ -115,8 +115,9 @@
         destVertexId.readFields(input);
         hasEdgeValue = input.readBoolean();
         if (hasEdgeValue) {
-            if (edgeValue == null)
+            if (edgeValue == null) {
                 edgeValue = (E) BspUtils.createEdgeValue(getConf());
+            }
             edgeValue.readFields(input);
         }
     }
@@ -128,8 +129,9 @@
         }
         destVertexId.write(output);
         output.writeBoolean(hasEdgeValue);
-        if (hasEdgeValue)
+        if (hasEdgeValue) {
             edgeValue.write(output);
+        }
     }
 
     @Override
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
index 734b1af..8d3d4c6 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MsgList.java
@@ -29,7 +29,7 @@
  */
 public class MsgList<M extends Writable> extends ArrayListWritable<M> {
     /** Defining a layout version for a serializable class. */
-    private static final long serialVersionUID = 100L;
+    private static final long serialVersionUID = 1L;
 
     /**
      * Default constructor.s
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 6856e9a..b7f9e3d 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -142,45 +142,6 @@
         usedValue = 0;
     }
 
-    private Edge<I, E> allocateEdge() {
-        Edge<I, E> edge;
-        if (usedEdge < edgePool.size()) {
-            edge = edgePool.get(usedEdge);
-            usedEdge++;
-        } else {
-            edge = new Edge<I, E>();
-            edgePool.add(edge);
-            usedEdge++;
-        }
-        return edge;
-    }
-
-    private M allocateMessage() {
-        M message;
-        if (usedMessage < msgPool.size()) {
-            message = msgPool.get(usedEdge);
-            usedMessage++;
-        } else {
-            message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
-            msgPool.add(message);
-            usedMessage++;
-        }
-        return message;
-    }
-
-    private V allocateValue() {
-        V value;
-        if (usedValue < valuePool.size()) {
-            value = valuePool.get(usedEdge);
-            usedValue++;
-        } else {
-            value = BspUtils.<V> createVertexValue(getContext().getConfiguration());
-            valuePool.add(value);
-            usedValue++;
-        }
-        return value;
-    }
-
     /**
      * Set the vertex id
      * 
@@ -201,60 +162,24 @@
     }
 
     /**
-     * Set the global superstep for all the vertices (internal use)
+     * Get the vertex value
      * 
-     * @param superstep
-     *            New superstep
+     * @return the vertex value
      */
-    public static void setSuperstep(long superstep) {
-        Vertex.superstep = superstep;
-    }
-
-    public static long getCurrentSuperstep() {
-        return superstep;
-    }
-
-    public final long getSuperstep() {
-        return superstep;
-    }
-
     public final V getVertexValue() {
         return vertexValue;
     }
 
+    /**
+     * Set the vertex value
+     * 
+     * @param vertexValue
+     */
     public final void setVertexValue(V vertexValue) {
         this.vertexValue = vertexValue;
         this.updated = true;
     }
 
-    /**
-     * Set the total number of vertices from the last superstep.
-     * 
-     * @param numVertices
-     *            Aggregate vertices in the last superstep
-     */
-    public static void setNumVertices(long numVertices) {
-        Vertex.numVertices = numVertices;
-    }
-
-    public final long getNumVertices() {
-        return numVertices;
-    }
-
-    /**
-     * Set the total number of edges from the last superstep.
-     * 
-     * @param numEdges
-     *            Aggregate edges in the last superstep
-     */
-    public static void setNumEdges(long numEdges) {
-        Vertex.numEdges = numEdges;
-    }
-
-    public final long getNumEdges() {
-        return numEdges;
-    }
-
     /***
      * Send a message to a specific vertex
      * 
@@ -309,6 +234,7 @@
         vertexId.readFields(in);
         delegate.setVertexId(vertexId);
         boolean hasVertexValue = in.readBoolean();
+
         if (hasVertexValue) {
             vertexValue = allocateValue();
             vertexValue.readFields(in);
@@ -352,12 +278,6 @@
         out.writeBoolean(halt);
     }
 
-    private boolean addEdge(Edge<I, E> edge) {
-        edge.setConf(getContext().getConfiguration());
-        destEdgeList.add(edge);
-        return true;
-    }
-
     /**
      * Get the list of incoming messages
      * 
@@ -376,14 +296,7 @@
         return this.destEdgeList;
     }
 
-    public final Mapper<?, ?, ?, ?>.Context getContext() {
-        return context;
-    }
-
-    public final static void setContext(Mapper<?, ?, ?, ?>.Context context) {
-        Vertex.context = context;
-    }
-
+    @Override
     @SuppressWarnings("unchecked")
     public String toString() {
         Collections.sort(destEdgeList);
@@ -396,37 +309,236 @@
         return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + ", edges=" + edgeBuffer + ")";
     }
 
-    public void setOutputWriters(List<IFrameWriter> writers) {
-        delegate.setOutputWriters(writers);
-    }
-
-    public void setOutputAppenders(List<FrameTupleAppender> appenders) {
-        delegate.setOutputAppenders(appenders);
-    }
-
-    public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
-        delegate.setOutputTupleBuilders(tbs);
-    }
-
-    public void finishCompute() throws IOException {
-        delegate.finishCompute();
-    }
-
-    public boolean hasUpdate() {
-        return this.updated;
-    }
-
-    public boolean hasMessage() {
-        return this.hasMessage;
-    }
-
+    /**
+     * Get the number of outgoing edges
+     * 
+     * @return the number of outging edges
+     */
     public int getNumOutEdges() {
         return destEdgeList.size();
     }
 
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputWriters(List<IFrameWriter> writers) {
+        delegate.setOutputWriters(writers);
+    }
+
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputAppenders(List<FrameTupleAppender> appenders) {
+        delegate.setOutputAppenders(appenders);
+    }
+
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
+        delegate.setOutputTupleBuilders(tbs);
+    }
+
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void finishCompute() throws IOException {
+        delegate.finishCompute();
+    }
+
+    /**
+     * Pregelix internal use only
+     */
+    public boolean hasUpdate() {
+        return this.updated;
+    }
+
+    /**
+     * Pregelix internal use only
+     */
+    public boolean hasMessage() {
+        return this.hasMessage;
+    }
+
+    /**
+     * sort the edges
+     */
     @SuppressWarnings("unchecked")
     public void sortEdges() {
-        Collections.sort((List) destEdgeList);
+        Collections.sort(destEdgeList);
+    }
+
+    /**
+     * Allocate a new edge from the edge pool
+     */
+    private Edge<I, E> allocateEdge() {
+        Edge<I, E> edge;
+        if (usedEdge < edgePool.size()) {
+            edge = edgePool.get(usedEdge);
+            usedEdge++;
+        } else {
+            edge = new Edge<I, E>();
+            edgePool.add(edge);
+            usedEdge++;
+        }
+        return edge;
+    }
+
+    /**
+     * Allocate a new message from the message pool
+     */
+    private M allocateMessage() {
+        M message;
+        if (usedMessage < msgPool.size()) {
+            message = msgPool.get(usedEdge);
+            usedMessage++;
+        } else {
+            message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
+            msgPool.add(message);
+            usedMessage++;
+        }
+        return message;
+    }
+
+    /**
+     * Set the global superstep for all the vertices (internal use)
+     * 
+     * @param superstep
+     *            New superstep
+     */
+    public static final void setSuperstep(long superstep) {
+        Vertex.superstep = superstep;
+    }
+
+    /**
+     * Add an outgoing edge into the vertex
+     * 
+     * @param edge
+     *            the edge to be added
+     * @return true if the edge list changed as a result of this call
+     */
+    public boolean addEdge(Edge<I, E> edge) {
+        edge.setConf(getContext().getConfiguration());
+        return destEdgeList.add(edge);
+    }
+
+    /**
+     * remove an outgoing edge in the graph
+     * 
+     * @param edge
+     *            the edge to be removed
+     * @return true if the edge is in the edge list of the vertex
+     */
+    public boolean removeEdge(Edge<I, E> edge) {
+        return destEdgeList.remove(edge);
+    }
+
+    /**
+     * Add a new vertex into the graph
+     * 
+     * @param vertexId the vertex id
+     * @param vertex the vertex
+     */
+    public final void addVertex(I vertexId, V vertex) {
+        delegate.addVertex(vertexId, vertex);
+    }
+
+    /**
+     * Delete a vertex from id
+     * 
+     * @param vertexId  the vertex id
+     */
+    public final void deleteVertex(I vertexId) {
+        delegate.deleteVertex(vertexId);
+    }
+
+    /**
+     * Allocate a vertex value from the object pool
+     * 
+     * @return a vertex value instance
+     */
+    private V allocateValue() {
+        V value;
+        if (usedValue < valuePool.size()) {
+            value = valuePool.get(usedValue);
+            usedValue++;
+        } else {
+            value = BspUtils.<V> createVertexValue(getContext().getConfiguration());
+            valuePool.add(value);
+            usedValue++;
+        }
+        return value;
+    }
+
+    /**
+     * Get the current global superstep number
+     * 
+     * @return the current superstep number
+     */
+    public static final long getSuperstep() {
+        return superstep;
+    }
+
+    /**
+     * Set the total number of vertices from the last superstep.
+     * 
+     * @param numVertices
+     *            Aggregate vertices in the last superstep
+     */
+    public static final void setNumVertices(long numVertices) {
+        Vertex.numVertices = numVertices;
+    }
+
+    /**
+     * Get the number of vertexes in the graph
+     * 
+     * @return the number of vertexes in the graph
+     */
+    public static final long getNumVertices() {
+        return numVertices;
+    }
+
+    /**
+     * Set the total number of edges from the last superstep.
+     * 
+     * @param numEdges
+     *            Aggregate edges in the last superstep
+     */
+    public static void setNumEdges(long numEdges) {
+        Vertex.numEdges = numEdges;
+    }
+
+    /**
+     * Get the number of edges from this graph
+     * 
+     * @return the number of edges in the graph
+     */
+    public static final long getNumEdges() {
+        return numEdges;
+    }
+
+    /**
+     * Pregelix internal use only
+     */
+    public static final Mapper<?, ?, ?, ?>.Context getContext() {
+        return context;
+    }
+
+    /**
+     * Pregelix internal use only
+     * 
+     * @param context
+     */
+    public static final void setContext(Mapper<?, ?, ?, ?>.Context context) {
+        Vertex.context = context;
     }
 
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
index 7267f30..d949bc5 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
@@ -44,6 +44,16 @@
     private IFrameWriter aliveWriter;
     private FrameTupleAppender appenderAlive;
 
+    /** the tuple for insert */
+    private ArrayTupleBuilder insertTb;
+    private IFrameWriter insertWriter;
+    private FrameTupleAppender appenderInsert;
+
+    /** the tuple for insert */
+    private ArrayTupleBuilder deleteTb;
+    private IFrameWriter deleteWriter;
+    private FrameTupleAppender appenderDelete;
+
     /** message list */
     private MsgList dummyMessageList = new MsgList();
     /** whether alive message should be pushed out */
@@ -95,25 +105,57 @@
         this.vertexId = vertexId;
     }
 
+    public final void addVertex(I vertexId, V vertex) {
+        try {
+            insertTb.reset();
+            DataOutput outputInsert = insertTb.getDataOutput();
+            vertexId.write(outputInsert);
+            insertTb.addFieldEndOffset();
+            vertex.write(outputInsert);
+            insertTb.addFieldEndOffset();
+            FrameTupleUtils.flushTuple(appenderInsert, insertTb, insertWriter);
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public final void deleteVertex(I vertexId) {
+        try {
+            deleteTb.reset();
+            DataOutput outputDelete = deleteTb.getDataOutput();
+            vertexId.write(outputDelete);
+            deleteTb.addFieldEndOffset();
+            FrameTupleUtils.flushTuple(appenderDelete, deleteTb, deleteWriter);
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
     public final void setOutputWriters(List<IFrameWriter> outputs) {
         msgWriter = outputs.get(0);
-        if (outputs.size() > 1) {
-            aliveWriter = outputs.get(1);
+        insertWriter = outputs.get(1);
+        deleteWriter = outputs.get(2);
+        if (outputs.size() > 3) {
+            aliveWriter = outputs.get(outputs.size() - 1);
             pushAlive = true;
         }
     }
 
     public final void setOutputAppenders(List<FrameTupleAppender> appenders) {
         appenderMsg = appenders.get(0);
-        if (appenders.size() > 1) {
-            appenderAlive = appenders.get(1);
+        appenderInsert = appenders.get(1);
+        appenderDelete = appenders.get(2);
+        if (appenders.size() > 3) {
+            appenderAlive = appenders.get(appenders.size() - 1);
         }
     }
 
     public final void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
         message = tbs.get(0);
-        if (tbs.size() > 1) {
-            alive = tbs.get(1);
+        insertTb = tbs.get(1);
+        deleteTb = tbs.get(2);
+        if (tbs.size() > 3) {
+            alive = tbs.get(tbs.size() - 1);
         }
     }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
index 7179737..ea33691 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/BasicGenInputSplit.java
@@ -21,60 +21,64 @@
 import java.io.Serializable;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 
 /**
  * This InputSplit will not give any ordering or location data. It is used
  * internally by BspInputFormat (which determines how many tasks to run the
  * application on). Users should not use this directly.
  */
-public class BasicGenInputSplit extends InputSplit implements Writable, Serializable {
-    private static final long serialVersionUID = 1L;
-    /** Number of splits */
-    private int numSplits = -1;
-    /** Split index */
-    private int splitIndex = -1;
+public class BasicGenInputSplit extends FileSplit implements Writable,
+		Serializable {
+	private static final long serialVersionUID = 1L;
+	/** Number of splits */
+	private int numSplits = -1;
+	/** Split index */
+	private int splitIndex = -1;
 
-    public BasicGenInputSplit() {
-    }
+	public BasicGenInputSplit() {
+		super(null, 0, 0, null);
+	}
 
-    public BasicGenInputSplit(int splitIndex, int numSplits) {
-        this.splitIndex = splitIndex;
-        this.numSplits = numSplits;
-    }
+	public BasicGenInputSplit(int splitIndex, int numSplits) {
+		super(null, 0, 0, null);
+		this.splitIndex = splitIndex;
+		this.numSplits = numSplits;
+	}
 
-    @Override
-    public long getLength() throws IOException, InterruptedException {
-        return 0;
-    }
+	@Override
+	public long getLength() {
+		return 0;
+	}
 
-    @Override
-    public String[] getLocations() throws IOException, InterruptedException {
-        return new String[] {};
-    }
+	@Override
+	public String[] getLocations() throws IOException {
+		return new String[] {};
+	}
 
-    @Override
-    public void readFields(DataInput in) throws IOException {
-        splitIndex = in.readInt();
-        numSplits = in.readInt();
-    }
+	@Override
+	public void readFields(DataInput in) throws IOException {
+		splitIndex = in.readInt();
+		numSplits = in.readInt();
+	}
 
-    @Override
-    public void write(DataOutput out) throws IOException {
-        out.writeInt(splitIndex);
-        out.writeInt(numSplits);
-    }
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeInt(splitIndex);
+		out.writeInt(numSplits);
+	}
 
-    public int getSplitIndex() {
-        return splitIndex;
-    }
+	public int getSplitIndex() {
+		return splitIndex;
+	}
 
-    public int getNumSplits() {
-        return numSplits;
-    }
+	public int getNumSplits() {
+		return numSplits;
+	}
 
-    @Override
-    public String toString() {
-        return "'" + getClass().getCanonicalName() + ", index=" + getSplitIndex() + ", num=" + getNumSplits();
-    }
+	@Override
+	public String toString() {
+		return "'" + getClass().getCanonicalName() + ", index="
+				+ getSplitIndex() + ", num=" + getNumSplits();
+	}
 }
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 6ef7e13..8b6d1b6 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
@@ -60,8 +60,12 @@
     public static final String NUM_VERTICE = "pregelix.numVertices";
     /** num of edges */
     public static final String NUM_EDGES = "pregelix.numEdges";
+    /** increase state length */
+    public static final String INCREASE_STATE_LENGTH = "pregelix.incStateLength";
     /** job id */
     public static final String JOB_ID = "pregelix.jobid";
+    /** frame size */
+    public static final String FRAME_SIZE = "pregelix.framesize";
 
     /**
      * Constructor that will instantiate the configuration
@@ -130,8 +134,8 @@
     /**
      * Set the global aggregator class (optional)
      * 
-     * @param vertexCombinerClass
-     *            Determines how vertex messages are combined
+     * @param globalAggregatorClass
+     *            Determines how messages are globally aggregated
      */
     final public void setGlobalAggregatorClass(Class<?> globalAggregatorClass) {
         getConfiguration().setClass(GLOBAL_AGGREGATOR_CLASS, globalAggregatorClass, GlobalAggregator.class);
@@ -139,11 +143,27 @@
 
     /**
      * Set the job Id
-     * 
-     * @param vertexCombinerClass
-     *            Determines how vertex messages are combined
      */
     final public void setJobId(String jobId) {
         getConfiguration().set(JOB_ID, jobId);
     }
+
+    /**
+     * Set whether the vertex state length can be dynamically increased
+     * 
+     * @param jobId
+     */
+    final public void setDynamicVertexValueSize(boolean incStateLengthDynamically) {
+        getConfiguration().setBoolean(INCREASE_STATE_LENGTH, incStateLengthDynamically);
+    }
+
+    /**
+     * Set the frame size for a job
+     * 
+     * @param frameSize
+     *            the desired frame size
+     */
+    final public void setFrameSize(int frameSize) {
+        getConfiguration().setInt(FRAME_SIZE, frameSize);
+    }
 }
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 7c4853f..ff9724d 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
@@ -410,4 +410,26 @@
             throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
         }
     }
+
+    /**
+     * Get the job configuration parameter whether the vertex states will increase dynamically
+     * 
+     * @param conf
+     *            the job configuration
+     * @return the boolean setting of the parameter, by default it is false
+     */
+    public static boolean getDynamicVertexValueSize(Configuration conf) {
+        return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, false);
+    }
+
+    /**
+     * Get the specified frame size
+     * 
+     * @param conf
+     *            the job configuration
+     * @return the specified frame size; -1 if it is not set by users
+     */
+    public static int getFrameSize(Configuration conf) {
+        return conf.getInt(PregelixJob.FRAME_SIZE, -1);
+    }
 }
diff --git a/pregelix/pregelix-core/pom.xml b/pregelix/pregelix-core/pom.xml
index 17ea6c3..34faa82 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.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/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<artifactId>pregelix-core</artifactId>
 	<packaging>jar</packaging>
@@ -19,6 +20,7 @@
 		<plugins>
 			<plugin>
 				<artifactId>maven-jar-plugin</artifactId>
+				<version>2.4</version>
 				<executions>
 					<execution>
 						<id>balancer</id>
@@ -38,11 +40,6 @@
 							</includes>
 						</configuration>
 					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
 					<execution>
 						<id>generator</id>
 						<goals>
@@ -64,25 +61,6 @@
 				</executions>
 			</plugin>
 			<plugin>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>patch</id>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<phase>package</phase>
-						<configuration>
-							<classifier>patch</classifier>
-							<finalName>a-hadoop</finalName>
-							<includes>
-								<include>**/org/apache/**</include>
-							</includes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-compiler-plugin</artifactId>
 				<version>2.0.2</version>
@@ -94,6 +72,7 @@
 			<plugin>
 				<groupId>org.codehaus.mojo</groupId>
 				<artifactId>appassembler-maven-plugin</artifactId>
+				<version>1.3</version>
 				<executions>
 					<execution>
 						<configuration>
@@ -166,25 +145,6 @@
 							</resources>
 						</configuration>
 					</execution>
-					<execution>
-						<id>copy-hadoop-patch</id>
-						<!-- here the phase you need -->
-						<phase>package</phase>
-						<goals>
-							<goal>copy-resources</goal>
-						</goals>
-						<configuration>
-							<outputDirectory>target/appassembler/lib</outputDirectory>
-							<resources>
-								<resource>
-									<directory>target</directory>
-									<includes>
-										<include>a-hadoop-patch.jar</include>
-									</includes>
-								</resource>
-							</resources>
-						</configuration>
-					</execution>
 				</executions>
 			</plugin>
 			<plugin>
@@ -193,7 +153,8 @@
 				<version>2.7.2</version>
 				<configuration>
 					<forkMode>pertest</forkMode>
-					<argLine>-enableassertions -Xmx512m -XX:MaxPermSize=300m -Dfile.encoding=UTF-8
+					<argLine>-enableassertions -Xmx512m -XX:MaxPermSize=300m
+						-Dfile.encoding=UTF-8
 						-Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
 					<includes>
 						<include>**/*TestSuite.java</include>
@@ -203,6 +164,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
@@ -335,7 +297,7 @@
 		<dependency>
 			<groupId>edu.uci.ics.hyracks.examples</groupId>
 			<artifactId>hyracks-integration-tests</artifactId>
-			<version>0.2.1</version>
+			<version>0.2.3-SNAPSHOT</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
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 de29dbc..0b1be61 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
@@ -76,8 +76,8 @@
 import edu.uci.ics.pregelix.core.util.DatatypeHelper;
 import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.FileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
@@ -89,8 +89,6 @@
     private static final Logger LOGGER = Logger.getLogger(JobGen.class.getName());
     protected static final int MB = 1048576;
     protected static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
-    protected static final int frameSize = ClusterConfig.getFrameSize();
-    protected static final int maxFrameSize = (int) (((long) 32 * MB) / frameSize);
     protected static final int tableSize = 10485767;
     protected static final String PRIMARY_INDEX = "primary";
     protected final Configuration conf;
@@ -98,6 +96,8 @@
     protected IIndexRegistryProvider<IIndex> treeRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
     protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
     protected String jobId = new UUID(System.currentTimeMillis(), System.nanoTime()).toString();
+    protected int frameSize = ClusterConfig.getFrameSize();
+    protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
 
     protected static final String SECONDARY_INDEX_ODD = "secondary1";
     protected static final String SECONDARY_INDEX_EVEN = "secondary2";
@@ -107,6 +107,17 @@
         this.giraphJob = job;
         this.initJobConfiguration();
         job.setJobId(jobId);
+
+        // 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;
+            maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
+        }
+        if (maxFrameNumber <= 0) {
+            maxFrameNumber = 1;
+        }
     }
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -189,9 +200,10 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
         VertexFileScanOperatorDescriptor scanner = new VertexFileScanOperatorDescriptor(spec, recordDescriptor, splits,
-                confFactory);
-        ClusterConfig.setLocationConstraint(spec, scanner, splits);
+                readSchedule, confFactory);
+        ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
          * construct sort operator
@@ -203,7 +215,7 @@
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
         comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameSize, sortFields,
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
         ClusterConfig.setLocationConstraint(spec, sorter);
 
@@ -264,9 +276,10 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
         VertexFileScanOperatorDescriptor scanner = new VertexFileScanOperatorDescriptor(spec, recordDescriptor, splits,
-                confFactory);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, scanner, splits.size());
+                readSchedule, confFactory);
+        ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
          * construct sort operator
@@ -280,7 +293,7 @@
                 .getClass());
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameLimit, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, sorter, splits.size());
+        ClusterConfig.setLocationConstraint(spec, sorter);
 
         /**
          * construct write file operator
@@ -292,7 +305,7 @@
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
                 vertexIdClass.getName(), vertexClass.getName());
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, inputRdFactory,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -357,7 +370,7 @@
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
                 vertexIdClass.getName(), vertexClass.getName());
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, inputRdFactory,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
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 00cdf07..727e7fe 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
@@ -39,6 +39,9 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -127,13 +130,16 @@
                 MsgList.class.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 4,
+                new BTreeDataflowHelperFactory(), inputRdFactory, 6,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdFinal);
+                rdPartialAggregate, rdInsert, rdDelete, rdFinal);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -173,7 +179,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
@@ -212,9 +218,36 @@
         EmptySinkOperatorDescriptor emptySink = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -223,7 +256,18 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 3, btreeBulkLoad, 0);
+
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
                 localGby, 0, globalGby, 0);
@@ -235,6 +279,8 @@
         spec.addRoot(btreeBulkLoad);
         spec.addRoot(terminateWriter);
         spec.addRoot(finalAggregator);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         spec.setFrameSize(frameSize);
@@ -261,6 +307,9 @@
                 .getClass());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -316,8 +365,8 @@
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 4, new ComputeUpdateFunctionFactory(confFactory),
-                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdFinal);
+                new BTreeDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
+                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
         ClusterConfig.setLocationConstraint(spec, join);
 
         /**
@@ -342,7 +391,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
@@ -395,6 +444,32 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
@@ -406,10 +481,18 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
                 terminateWriter, 0);
-
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), join, 3, btreeBulkLoad, 0);
+
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), join, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
                 localGby, 0, globalGby, 0);
@@ -420,6 +503,8 @@
         spec.addRoot(emptySink);
         spec.addRoot(btreeBulkLoad);
         spec.addRoot(terminateWriter);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         spec.setFrameSize(frameSize);
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 3847aa7..9bad169 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
@@ -39,6 +39,9 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -121,13 +124,16 @@
                 vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate);
+                rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -139,14 +145,15 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
          * construct local pre-clustered group-by operator
          */
-        IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false, false);
+        IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false,
+                false);
         PreclusteredGroupOperatorDescriptor localGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localGby);
@@ -156,8 +163,8 @@
          */
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
-        IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils
-                .getAccumulatingAggregatorFactory(conf, true, true);
+        IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(conf,
+                true, true);
         PreclusteredGroupOperatorDescriptor globalGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 sortCmpFactories, aggregatorFactoryFinal, rdFinal);
         ClusterConfig.setLocationConstraint(spec, globalGby);
@@ -193,6 +200,33 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
         /** connect all operators **/
@@ -203,6 +237,20 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
+
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
+        /**
+         * connect the group-by operator
+         */
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
                 localGby, 0, globalGby, 0);
@@ -213,6 +261,8 @@
         spec.addRoot(terminateWriter);
         spec.addRoot(finalAggregator);
         spec.addRoot(emptySink2);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         spec.setFrameSize(frameSize);
@@ -239,6 +289,9 @@
                 .getClass());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -286,9 +339,9 @@
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
                 new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate);
+                rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, join);
 
         /**
@@ -299,14 +352,15 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
          * construct local pre-clustered group-by operator
          */
-        IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false, false);
+        IAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf, false,
+                false);
         PreclusteredGroupOperatorDescriptor localGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localGby);
@@ -314,8 +368,8 @@
         /**
          * construct global group-by operator
          */
-        IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils
-                .getAccumulatingAggregatorFactory(conf, true, true);
+        IAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(conf,
+                true, true);
         PreclusteredGroupOperatorDescriptor globalGby = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 sortCmpFactories, aggregatorFactoryFinal, rdFinal);
         ClusterConfig.setLocationConstraint(spec, globalGby);
@@ -351,6 +405,33 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
@@ -364,6 +445,15 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
+
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
                 localGby, 0, globalGby, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index ec783a7..ffdef10 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
@@ -38,6 +38,9 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -123,10 +126,14 @@
                 vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
                 rdPartialAggregate);
         ClusterConfig.setLocationConstraint(spec, scanner);
@@ -140,7 +147,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
@@ -185,6 +192,33 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
@@ -196,6 +230,17 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
+
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
@@ -204,6 +249,8 @@
         spec.addRoot(terminateWriter);
         spec.addRoot(finalAggregator);
         spec.addRoot(emptySink2);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setConnectorPolicyAssignmentPolicy(new NonCombinerConnectorPolicyAssignmentPolicy());
         spec.setFrameSize(frameSize);
@@ -230,6 +277,9 @@
                 .getClass());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -277,7 +327,7 @@
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
                 new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
                 rdPartialAggregate);
         ClusterConfig.setLocationConstraint(spec, join);
@@ -290,7 +340,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
@@ -333,6 +383,31 @@
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
+        
+
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+        
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
@@ -347,6 +422,16 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+        
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index bb939e3..cc12523 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
@@ -38,6 +38,9 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -119,10 +122,14 @@
                 vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 3,
+                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
                 rdPartialAggregate);
         ClusterConfig.setLocationConstraint(spec, scanner);
@@ -136,7 +143,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
@@ -152,7 +159,7 @@
         /**
          * construct global sort operator
          */
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
@@ -198,6 +205,33 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
         /** connect all operators **/
@@ -208,6 +242,16 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
+                0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
@@ -218,6 +262,8 @@
         spec.addRoot(terminateWriter);
         spec.addRoot(finalAggregator);
         spec.addRoot(emptySink2);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setFrameSize(frameSize);
         return spec;
@@ -243,6 +289,9 @@
                 .getClass());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                vertexClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -290,7 +339,7 @@
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 3,
+                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
                 new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
                 rdPartialAggregate);
         ClusterConfig.setLocationConstraint(spec, join);
@@ -303,7 +352,7 @@
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
@@ -319,7 +368,7 @@
         /**
          * construct global sort operator
          */
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameSize, keyFields,
+        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
                 nkmFactory, sortCmpFactories, rdUnnestedMessage);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
@@ -363,6 +412,33 @@
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
 
+        /**
+         * add the insert operator to insert vertexes
+         */
+        int[] fieldPermutation = new int[] { 0, 1 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, insertOp);
+
+        /**
+         * add the delete operator to delete vertexes
+         */
+        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
+                NoOpOperationCallbackProvider.INSTANCE);
+        ClusterConfig.setLocationConstraint(spec, deleteOp);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink3);
+
+        /** construct empty sink operator */
+        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
+        ClusterConfig.setLocationConstraint(spec, emptySink4);
+
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
@@ -376,6 +452,14 @@
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
+        /**
+         * connect the insert/delete operator
+         */
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
+
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
@@ -386,6 +470,8 @@
         spec.addRoot(terminateWriter);
         spec.addRoot(finalAggregator);
         spec.addRoot(emptySink);
+        spec.addRoot(emptySink3);
+        spec.addRoot(emptySink4);
 
         spec.setFrameSize(frameSize);
         return spec;
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 8eadab9..d26e637 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
@@ -40,6 +40,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.hdfs2.scheduler.Scheduler;
 
 public class ClusterConfig {
 
@@ -49,6 +50,7 @@
     private static Properties clusterProperties = new Properties();
     private static Map<String, List<String>> ipToNcMapping;
     private static String[] stores;
+    private static Scheduler hdfsScheduler;
 
     /**
      * let tests set config path to be whatever
@@ -211,6 +213,8 @@
                 NCs[i] = entry.getKey();
                 i++;
             }
+
+            hdfsScheduler = new Scheduler(ipAddress, port);
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
@@ -218,4 +222,8 @@
         loadClusterProperties();
         loadStores();
     }
+
+    public static Scheduler getHdfsScheduler() {
+        return hdfsScheduler;
+    }
 }
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 ed04746..2a2e2bf 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
@@ -51,7 +51,8 @@
         ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
         ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
         ccConfig.defaultMaxJobAttempts = 0;
-        ccConfig.jobHistorySize = 10;
+        ccConfig.jobHistorySize = 0;
+        ccConfig.profileDumpPeriod = -1;
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
diff --git a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java b/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java
deleted file mode 100644
index 5efdde8..0000000
--- a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/fs/Path.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Names a file or directory in a {@link FileSystem}. Path strings use slash as
- * the directory separator. A path string is absolute if it begins with a slash.
- */
-@SuppressWarnings("rawtypes")
-public class Path implements Comparable, Serializable {
-    private static final long serialVersionUID = 1L;
-    /** The directory separator, a slash. */
-    public static final String SEPARATOR = "/";
-    public static final char SEPARATOR_CHAR = '/';
-
-    public static final String CUR_DIR = ".";
-
-    static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
-
-    private URI uri; // a hierarchical uri
-
-    /** Resolve a child path against a parent path. */
-    public Path(String parent, String child) {
-        this(new Path(parent), new Path(child));
-    }
-
-    /** Resolve a child path against a parent path. */
-    public Path(Path parent, String child) {
-        this(parent, new Path(child));
-    }
-
-    /** Resolve a child path against a parent path. */
-    public Path(String parent, Path child) {
-        this(new Path(parent), child);
-    }
-
-    /** Resolve a child path against a parent path. */
-    public Path(Path parent, Path child) {
-        // Add a slash to parent's path so resolution is compatible with URI's
-        URI parentUri = parent.uri;
-        String parentPath = parentUri.getPath();
-        if (!(parentPath.equals("/") || parentPath.equals("")))
-            try {
-                parentUri = new URI(parentUri.getScheme(), parentUri.getAuthority(), parentUri.getPath() + "/", null,
-                        parentUri.getFragment());
-            } catch (URISyntaxException e) {
-                throw new IllegalArgumentException(e);
-            }
-        URI resolved = parentUri.resolve(child.uri);
-        initialize(resolved.getScheme(), resolved.getAuthority(), normalizePath(resolved.getPath()),
-                resolved.getFragment());
-    }
-
-    private void checkPathArg(String path) {
-        // disallow construction of a Path from an empty string
-        if (path == null) {
-            throw new IllegalArgumentException("Can not create a Path from a null string");
-        }
-        if (path.length() == 0) {
-            throw new IllegalArgumentException("Can not create a Path from an empty string");
-        }
-    }
-
-    /**
-     * Construct a path from a String. Path strings are URIs, but with unescaped
-     * elements and some additional normalization.
-     */
-    public Path(String pathString) {
-        checkPathArg(pathString);
-
-        // We can't use 'new URI(String)' directly, since it assumes things are
-        // escaped, which we don't require of Paths.
-
-        // add a slash in front of paths with Windows drive letters
-        if (hasWindowsDrive(pathString, false))
-            pathString = "/" + pathString;
-
-        // parse uri components
-        String scheme = null;
-        String authority = null;
-
-        int start = 0;
-
-        // parse uri scheme, if any
-        int colon = pathString.indexOf(':');
-        int slash = pathString.indexOf('/');
-        if ((colon != -1) && ((slash == -1) || (colon < slash))) { // has a
-                                                                   // scheme
-            scheme = pathString.substring(0, colon);
-            start = colon + 1;
-        }
-
-        // parse uri authority, if any
-        if (pathString.startsWith("//", start) && (pathString.length() - start > 2)) { // has
-                                                                                       // authority
-            int nextSlash = pathString.indexOf('/', start + 2);
-            int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
-            authority = pathString.substring(start + 2, authEnd);
-            start = authEnd;
-        }
-
-        // uri path is the rest of the string -- query & fragment not supported
-        String path = pathString.substring(start, pathString.length());
-
-        initialize(scheme, authority, path, null);
-    }
-
-    /** Construct a Path from components. */
-    public Path(String scheme, String authority, String path) {
-        checkPathArg(path);
-        initialize(scheme, authority, path, null);
-    }
-
-    /**
-     * Construct a path from a URI
-     */
-    public Path(URI aUri) {
-        uri = aUri;
-    }
-
-    private void initialize(String scheme, String authority, String path, String fragment) {
-        try {
-            this.uri = new URI(scheme, authority, normalizePath(path), null, fragment).normalize();
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException(e);
-        }
-    }
-
-    private String normalizePath(String path) {
-        // remove double slashes & backslashes
-        if (path.indexOf("//") != -1) {
-            path = path.replace("//", "/");
-        }
-        if (path.indexOf("\\") != -1) {
-            path = path.replace("\\", "/");
-        }
-
-        // trim trailing slash from non-root path (ignoring windows drive)
-        int minLength = hasWindowsDrive(path, true) ? 4 : 1;
-        if (path.length() > minLength && path.endsWith("/")) {
-            path = path.substring(0, path.length() - 1);
-        }
-
-        return path;
-    }
-
-    private boolean hasWindowsDrive(String path, boolean slashed) {
-        if (!WINDOWS)
-            return false;
-        int start = slashed ? 1 : 0;
-        return path.length() >= start + 2
-                && (slashed ? path.charAt(0) == '/' : true)
-                && path.charAt(start + 1) == ':'
-                && ((path.charAt(start) >= 'A' && path.charAt(start) <= 'Z') || (path.charAt(start) >= 'a' && path
-                        .charAt(start) <= 'z'));
-    }
-
-    /** Convert this to a URI. */
-    public URI toUri() {
-        return uri;
-    }
-
-    /** Return the FileSystem that owns this Path. */
-    public FileSystem getFileSystem(Configuration conf) throws IOException {
-        return FileSystem.get(this.toUri(), conf);
-    }
-
-    /** True if the directory of this path is absolute. */
-    public boolean isAbsolute() {
-        int start = hasWindowsDrive(uri.getPath(), true) ? 3 : 0;
-        return uri.getPath().startsWith(SEPARATOR, start);
-    }
-
-    /** Returns the final component of this path. */
-    public String getName() {
-        String path = uri.getPath();
-        int slash = path.lastIndexOf(SEPARATOR);
-        return path.substring(slash + 1);
-    }
-
-    /** Returns the parent of a path or null if at root. */
-    public Path getParent() {
-        String path = uri.getPath();
-        int lastSlash = path.lastIndexOf('/');
-        int start = hasWindowsDrive(path, true) ? 3 : 0;
-        if ((path.length() == start) || // empty path
-                (lastSlash == start && path.length() == start + 1)) { // at root
-            return null;
-        }
-        String parent;
-        if (lastSlash == -1) {
-            parent = CUR_DIR;
-        } else {
-            int end = hasWindowsDrive(path, true) ? 3 : 0;
-            parent = path.substring(0, lastSlash == end ? end + 1 : lastSlash);
-        }
-        return new Path(uri.getScheme(), uri.getAuthority(), parent);
-    }
-
-    /** Adds a suffix to the final name in the path. */
-    public Path suffix(String suffix) {
-        return new Path(getParent(), getName() + suffix);
-    }
-
-    public String toString() {
-        // we can't use uri.toString(), which escapes everything, because we
-        // want
-        // illegal characters unescaped in the string, for glob processing, etc.
-        StringBuffer buffer = new StringBuffer();
-        if (uri.getScheme() != null) {
-            buffer.append(uri.getScheme());
-            buffer.append(":");
-        }
-        if (uri.getAuthority() != null) {
-            buffer.append("//");
-            buffer.append(uri.getAuthority());
-        }
-        if (uri.getPath() != null) {
-            String path = uri.getPath();
-            if (path.indexOf('/') == 0 && hasWindowsDrive(path, true) && // has
-                                                                         // windows
-                                                                         // drive
-                    uri.getScheme() == null && // but no scheme
-                    uri.getAuthority() == null) // or authority
-                path = path.substring(1); // remove slash before drive
-            buffer.append(path);
-        }
-        if (uri.getFragment() != null) {
-            buffer.append("#");
-            buffer.append(uri.getFragment());
-        }
-        return buffer.toString();
-    }
-
-    public boolean equals(Object o) {
-        if (!(o instanceof Path)) {
-            return false;
-        }
-        Path that = (Path) o;
-        return this.uri.equals(that.uri);
-    }
-
-    public int hashCode() {
-        return uri.hashCode();
-    }
-
-    public int compareTo(Object o) {
-        Path that = (Path) o;
-        return this.uri.compareTo(that.uri);
-    }
-
-    /** Return the number of elements in this path. */
-    public int depth() {
-        String path = uri.getPath();
-        int depth = 0;
-        int slash = path.length() == 1 && path.charAt(0) == '/' ? -1 : 0;
-        while (slash != -1) {
-            depth++;
-            slash = path.indexOf(SEPARATOR, slash + 1);
-        }
-        return depth;
-    }
-
-    /** Returns a qualified path object. */
-    public Path makeQualified(FileSystem fs) {
-        Path path = this;
-        if (!isAbsolute()) {
-            path = new Path(fs.getWorkingDirectory(), this);
-        }
-
-        URI pathUri = path.toUri();
-        URI fsUri = fs.getUri();
-
-        String scheme = pathUri.getScheme();
-        String authority = pathUri.getAuthority();
-        String fragment = pathUri.getFragment();
-        if (scheme != null && (authority != null || fsUri.getAuthority() == null))
-            return path;
-
-        if (scheme == null) {
-            scheme = fsUri.getScheme();
-        }
-
-        if (authority == null) {
-            authority = fsUri.getAuthority();
-            if (authority == null) {
-                authority = "";
-            }
-        }
-
-        URI newUri = null;
-        try {
-            newUri = new URI(scheme, authority, normalizePath(pathUri.getPath()), null, fragment);
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException(e);
-        }
-        return new Path(newUri);
-    }
-
-    /** Returns a qualified path object. */
-    public Path makeQualified(URI defaultUri, Path workingDir) {
-        Path path = this;
-        if (!isAbsolute()) {
-            path = new Path(workingDir, this);
-        }
-
-        URI pathUri = path.toUri();
-
-        String scheme = pathUri.getScheme();
-        String authority = pathUri.getAuthority();
-        String fragment = pathUri.getFragment();
-
-        if (scheme != null && (authority != null || defaultUri.getAuthority() == null))
-            return path;
-
-        if (scheme == null) {
-            scheme = defaultUri.getScheme();
-        }
-
-        if (authority == null) {
-            authority = defaultUri.getAuthority();
-            if (authority == null) {
-                authority = "";
-            }
-        }
-
-        URI newUri = null;
-        try {
-            newUri = new URI(scheme, authority, normalizePath(pathUri.getPath()), null, fragment);
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException(e);
-        }
-        return new Path(newUri);
-    }
-}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java b/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java
deleted file mode 100644
index ac72160..0000000
--- a/pregelix/pregelix-core/src/main/java/org/apache/hadoop/mapreduce/InputSplit.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapreduce;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * <code>InputSplit</code> represents the data to be processed by an individual {@link Mapper}.
- * <p>
- * Typically, it presents a byte-oriented view on the input and is the responsibility of {@link RecordReader} of the job to process this and present a record-oriented view.
- * 
- * @see InputFormat
- * @see RecordReader
- */
-public abstract class InputSplit implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    /**
-     * Get the size of the split, so that the input splits can be sorted by
-     * size.
-     * 
-     * @return the number of bytes in the split
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public abstract long getLength() throws IOException, InterruptedException;
-
-    /**
-     * Get the list of nodes by name where the data for the split would be
-     * local. The locations do not need to be serialized.
-     * 
-     * @return a new array of the node nodes.
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public abstract String[] getLocations() throws IOException, InterruptedException;
-}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelix b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
index c3fd27b..6997078 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelix
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
@@ -91,7 +91,7 @@
   REPO="$BASEDIR"/lib
 fi
 
-CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:"$BASEDIR"/etc:$(echo ${REPO}/*.jar | tr ' ' ':'):$1
+CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:"$BASEDIR"/etc:$1
 
 # For Cygwin, switch paths to Windows format before running java
 if $cygwin; then
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 4dfe57d..97659d4 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -62,7 +62,7 @@
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
 import edu.uci.ics.pregelix.core.util.TestUtils;
-import edu.uci.ics.pregelix.dataflow.std.FileWriteOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.ProjectOperatorDescriptor;
 import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
@@ -195,7 +195,7 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -362,7 +362,7 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -459,7 +459,7 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
@@ -566,7 +566,7 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        FileWriteOperatorDescriptor writer = new FileWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
diff --git a/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml b/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
index 1b9a4d6..f89dd79 100644
--- a/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
+++ b/pregelix/pregelix-core/src/test/resources/hadoop/conf/mapred-site.xml
@@ -18,8 +18,8 @@
       <value>20</value>
    </property>
    <property>
-      <name>mapred.min.split.size</name>
-      <value>65536</value>
+      <name>mapred.max.split.size</name>
+      <value>4096</value>
    </property>
 
 </configuration>
diff --git a/pregelix/pregelix-dataflow-std-base/pom.xml b/pregelix/pregelix-dataflow-std-base/pom.xml
index 53d7c22..05a1508 100644
--- a/pregelix/pregelix-dataflow-std-base/pom.xml
+++ b/pregelix/pregelix-dataflow-std-base/pom.xml
@@ -42,6 +42,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
index 62f92dd..a0d365f 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
@@ -16,17 +16,19 @@
 package edu.uci.ics.pregelix.dataflow.std.base;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public interface IUpdateFunction extends IFunction {
 
-    /**
-     * update the tuple pointed by tupleRef called after process,
-     * one-input-tuple-at-a-time
-     * 
-     * @param tupleRef
-     * @throws HyracksDataException
-     */
-    public void update(ITupleReference tupleRef) throws HyracksDataException;
+	/**
+	 * update the tuple pointed by tupleRef called after process,
+	 * one-input-tuple-at-a-time
+	 * 
+	 * @param tupleRef
+	 * @throws HyracksDataException
+	 */
+	public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb)
+			throws HyracksDataException;
 
 }
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index cc9a184..6c039bf 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/pom.xml
@@ -42,6 +42,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
@@ -143,5 +144,12 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-hdfs</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
 	</dependencies>
 </project>
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
index fb84aa0..3938613 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
@@ -43,6 +43,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class BTreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     protected TreeIndexDataflowHelper treeIndexHelper;
@@ -70,6 +71,8 @@
 
     private final IFrameWriter[] writers;
     private final FunctionProxy functionProxy;
+    private ArrayTupleBuilder cloneUpdateTb;
+    private final UpdateBuffer updateBuffer;
 
     public BTreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -94,6 +97,7 @@
         this.writers = new IFrameWriter[outputArity];
         this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
                 writers);
+        this.updateBuffer = new UpdateBuffer(ctx, 2);
     }
 
     @Override
@@ -122,6 +126,9 @@
             appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
             indexAccessor = btree.createAccessor();
+
+            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+            updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
             treeIndexHelper.deinit();
             throw new HyracksDataException(e);
@@ -136,7 +143,24 @@
         while (cursor.hasNext()) {
             cursor.next();
             ITupleReference tuple = cursor.getTuple();
-            functionProxy.functionCall(tuple);
+            functionProxy.functionCall(tuple, cloneUpdateTb);
+
+            //doing clone update
+            if (cloneUpdateTb.getSize() > 0) {
+                if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                    //release the cursor/latch
+                    cursor.close();
+                    //batch update
+                    updateBuffer.updateBTree(indexAccessor);
+
+                    //search again
+                    cursor.reset();
+                    rangePred.setLowKey(tuple, true);
+                    rangePred.setHighKey(highKey, highKeyInclusive);
+                    indexAccessor.search(cursor, rangePred);
+                }
+            }
+            cloneUpdateTb.reset();
         }
     }
 
@@ -168,6 +192,8 @@
         try {
             try {
                 cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
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 75a8087..37029f3 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
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.pregelix.dataflow.std;
 
-import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
@@ -44,6 +43,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     private TreeIndexDataflowHelper treeIndexOpHelper;
@@ -51,9 +51,6 @@
 
     private ByteBuffer writeBuffer;
     private FrameTupleAppender appender;
-    private ArrayTupleBuilder tb;
-    private DataOutput dos;
-
     private BTree btree;
     private PermutingFrameTupleReference lowKey;
     private PermutingFrameTupleReference highKey;
@@ -67,17 +64,16 @@
     protected ITreeIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
-    private final RecordDescriptor inputRecDesc;
-
     private final IFrameWriter[] writers;
     private final FunctionProxy functionProxy;
+    private ArrayTupleBuilder cloneUpdateTb;
+    private final UpdateBuffer updateBuffer;
 
     public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
             IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
-        inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.lowKeyInclusive = lowKeyInclusive;
@@ -95,6 +91,7 @@
         this.writers = new IFrameWriter[outputArity];
         this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
                 writers);
+        this.updateBuffer = new UpdateBuffer(ctx, 2);
     }
 
     protected void setCursor() {
@@ -144,12 +141,12 @@
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
             writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
-            dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
             indexAccessor = btree.createAccessor();
+            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+            updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
@@ -158,27 +155,29 @@
 
     private void writeSearchResults(IFrameTupleAccessor leftAccessor, int tIndex) throws Exception {
         while (cursor.hasNext()) {
-            tb.reset();
             cursor.next();
-
             ITupleReference tupleRef = cursor.getTuple();
-            for (int i = 0; i < inputRecDesc.getFields().length; i++) {
-                int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
-                int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
-                int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
-                int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
-                dos.write(leftAccessor.getBuffer().array(), offset, len);
-                tb.addFieldEndOffset();
-            }
-            for (int i = 0; i < tupleRef.getFieldCount(); i++) {
-                dos.write(tupleRef.getFieldData(i), tupleRef.getFieldStart(i), tupleRef.getFieldLength(i));
-                tb.addFieldEndOffset();
-            }
 
             /**
              * call the update function
              */
-            functionProxy.functionCall(tb, tupleRef);
+            functionProxy.functionCall(leftAccessor, tIndex, tupleRef, cloneUpdateTb);
+
+            if (cloneUpdateTb.getSize() > 0) {
+                if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                    //release the cursor/latch
+                    cursor.close();
+                    //batch update
+                    updateBuffer.updateBTree(indexAccessor);
+
+                    //search again
+                    cursor.reset();
+                    rangePred.setLowKey(tupleRef, true);
+                    rangePred.setHighKey(highKey, highKeyInclusive);
+                    indexAccessor.search(cursor, rangePred);
+                }
+            }
+            cloneUpdateTb.reset();
         }
     }
 
@@ -210,6 +209,8 @@
         try {
             try {
                 cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
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 c31ebd4..f7b3d62 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
@@ -45,6 +45,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
         AbstractUnaryInputOperatorNodePushable {
@@ -53,7 +54,7 @@
 
     private ByteBuffer writeBuffer;
     private FrameTupleAppender appender;
-    private ArrayTupleBuilder tb;
+    private ArrayTupleBuilder nullTupleBuilder;
     private DataOutput dos;
 
     private BTree btree;
@@ -76,6 +77,8 @@
 
     private final IFrameWriter[] writers;
     private final FunctionProxy functionProxy;
+    private ArrayTupleBuilder cloneUpdateTb;
+    private UpdateBuffer updateBuffer;
 
     public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -100,6 +103,7 @@
         this.writers = new IFrameWriter[outputArity];
         this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
                 writers);
+        this.updateBuffer = new UpdateBuffer(ctx, 2);
     }
 
     protected void setCursor() {
@@ -144,8 +148,15 @@
             rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
 
             writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
-            dos = tb.getDataOutput();
+
+            nullTupleBuilder = new ArrayTupleBuilder(inputRecDesc.getFields().length);
+            dos = nullTupleBuilder.getDataOutput();
+            nullTupleBuilder.reset();
+            for (int i = 0; i < inputRecDesc.getFields().length; i++) {
+                nullWriter[i].writeNull(dos);
+                nullTupleBuilder.addFieldEndOffset();
+            }
+
             appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
@@ -164,32 +175,38 @@
                 match = false;
             }
 
+            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+            updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
         }
     }
 
+    //for the join match casesos
     private void writeResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple)
             throws Exception {
-        tb.reset();
-        for (int i = 0; i < inputRecDesc.getFields().length; i++) {
-            int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
-            int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
-            int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
-            int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
-            dos.write(leftAccessor.getBuffer().array(), offset, len);
-            tb.addFieldEndOffset();
-        }
-        for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-            dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
-            tb.addFieldEndOffset();
-        }
-
         /**
          * function call
          */
-        functionProxy.functionCall(tb, frameTuple);
+        functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb);
+
+        //doing clone update
+        if (cloneUpdateTb.getSize() > 0) {
+            if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                //release the cursor/latch
+                cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
+
+                //search again and recover the cursor
+                cursor.reset();
+                rangePred.setLowKey(frameTuple, true);
+                rangePred.setHighKey(null, true);
+                indexAccessor.search(cursor, rangePred);
+            }
+            cloneUpdateTb.reset();
+        }
     }
 
     @Override
@@ -243,6 +260,8 @@
             }
             try {
                 cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
@@ -271,20 +290,27 @@
 
     /** write result for outer case */
     private void writeResults(ITupleReference frameTuple) throws Exception {
-        tb.reset();
-        for (int i = 0; i < inputRecDesc.getFields().length; i++) {
-            nullWriter[i].writeNull(dos);
-            tb.addFieldEndOffset();
-        }
-        for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-            dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
-            tb.addFieldEndOffset();
-        }
-
         /**
          * function call
          */
-        functionProxy.functionCall(tb, frameTuple);
+        functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb);
+
+        //doing clone update
+        if (cloneUpdateTb.getSize() > 0) {
+            if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                //release the cursor/latch
+                cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
+
+                //search again and recover the cursor
+                cursor.reset();
+                rangePred.setLowKey(frameTuple, true);
+                rangePred.setHighKey(null, true);
+                indexAccessor.search(cursor, rangePred);
+            }
+            cloneUpdateTb.reset();
+        }
     }
 
     @Override
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 0a966b5..6af60a8 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
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.pregelix.dataflow.std;
 
-import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
@@ -44,6 +43,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     private TreeIndexDataflowHelper treeIndexOpHelper;
@@ -51,8 +51,6 @@
 
     private ByteBuffer writeBuffer;
     private FrameTupleAppender appender;
-    private ArrayTupleBuilder tb;
-    private DataOutput dos;
 
     private BTree btree;
     private boolean isForward;
@@ -63,8 +61,6 @@
     protected ITreeIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
-    private final RecordDescriptor inputRecDesc;
-
     private PermutingFrameTupleReference lowKey;
     private PermutingFrameTupleReference highKey;
 
@@ -73,13 +69,14 @@
 
     private final IFrameWriter[] writers;
     private final FunctionProxy functionProxy;
+    private ArrayTupleBuilder cloneUpdateTb;
+    private UpdateBuffer updateBuffer;
 
     public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
             IRecordDescriptorFactory inputRdFactory, int outputArity) {
-        inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.isForward = isForward;
@@ -97,6 +94,7 @@
         this.writers = new IFrameWriter[outputArity];
         this.functionProxy = new FunctionProxy(ctx, functionFactory, preHookFactory, postHookFactory, inputRdFactory,
                 writers);
+        this.updateBuffer = new UpdateBuffer(ctx, 2);
     }
 
     protected void setCursor() {
@@ -123,8 +121,6 @@
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
             writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(btree.getFieldCount());
-            dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
@@ -142,7 +138,8 @@
                 currentTopTuple = cursor.getTuple();
                 match = false;
             }
-
+            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
+            updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
@@ -207,6 +204,9 @@
             }
             try {
                 cursor.close();
+
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
@@ -231,29 +231,47 @@
 
     /** write the right result */
     private void writeRightResults(ITupleReference frameTuple) throws Exception {
-        tb.reset();
-        for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-            dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
-            tb.addFieldEndOffset();
-        }
+        functionProxy.functionCall(frameTuple, cloneUpdateTb);
 
-        functionProxy.functionCall(tb, frameTuple);
+        //doing clone update
+        if (cloneUpdateTb.getSize() > 0) {
+            if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                //release the cursor/latch
+                cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
+
+                //search again
+                cursor.reset();
+                rangePred.setLowKey(frameTuple, true);
+                rangePred.setHighKey(null, true);
+                indexAccessor.search(cursor, rangePred);
+            }
+            cloneUpdateTb.reset();
+        }
     }
 
     /** write the left result */
     private void writeLeftResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple)
             throws Exception {
-        tb.reset();
-        for (int i = 0; i < inputRecDesc.getFields().length; i++) {
-            int tupleStart = leftAccessor.getTupleStartOffset(tIndex);
-            int fieldStart = leftAccessor.getFieldStartOffset(tIndex, i);
-            int offset = leftAccessor.getFieldSlotsLength() + tupleStart + fieldStart;
-            int len = leftAccessor.getFieldEndOffset(tIndex, i) - fieldStart;
-            dos.write(leftAccessor.getBuffer().array(), offset, len);
-            tb.addFieldEndOffset();
-        }
+        functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb);
 
-        functionProxy.functionCall(tb, frameTuple);
+        //doing clone update
+        if (cloneUpdateTb.getSize() > 0) {
+            if (!updateBuffer.appendTuple(cloneUpdateTb)) {
+                //release the cursor/latch
+                cursor.close();
+                //batch update
+                updateBuffer.updateBTree(indexAccessor);
+
+                //search again
+                cursor.reset();
+                rangePred.setLowKey(frameTuple, true);
+                rangePred.setHighKey(null, true);
+                indexAccessor.search(cursor, rangePred);
+            }
+            cloneUpdateTb.reset();
+        }
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
index 4b0f4a5..82ac18e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.pregelix.dataflow.util;
 
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -68,16 +69,19 @@
     /**
      * Call the function
      * 
-     * @param tb
-     *            input data
+     * @param leftAccessor
+     *            input page accessor
+     * @param leftTupleIndex
+     *            the tuple index in the page
      * @param updateRef
      *            update pointer
      * @throws HyracksDataException
      */
-    public void functionCall(ArrayTupleBuilder tb, ITupleReference updateRef) throws HyracksDataException {
-        Object[] tuple = tupleDe.deserializeRecord(tb);
+    public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, ITupleReference right,
+            ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+        Object[] tuple = tupleDe.deserializeRecord(leftAccessor, leftTupleIndex, right);
         function.process(tuple);
-        function.update(updateRef);
+        function.update(right, cloneUpdateTb);
     }
 
     /**
@@ -86,10 +90,26 @@
      * @param updateRef
      * @throws HyracksDataException
      */
-    public void functionCall(ITupleReference updateRef) throws HyracksDataException {
+    public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
         Object[] tuple = tupleDe.deserializeRecord(updateRef);
         function.process(tuple);
-        function.update(updateRef);
+        function.update(updateRef, cloneUpdateTb);
+    }
+
+    /**
+     * Call the function
+     * 
+     * @param tb
+     *            input data
+     * @param inPlaceUpdateRef
+     *            update pointer
+     * @throws HyracksDataException
+     */
+    public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb)
+            throws HyracksDataException {
+        Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef);
+        function.process(tuple);
+        function.update(inPlaceUpdateRef, cloneUpdateTb);
     }
 
     /**
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
index 5ae1d81..4fe83db 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
@@ -20,6 +20,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -42,7 +43,7 @@
     }
 
     public Object[] deserializeRecord(ITupleReference tupleRef) throws HyracksDataException {
-        for (int i = 0; i < record.length; ++i) {
+        for (int i = 0; i < tupleRef.getFieldCount(); ++i) {
             byte[] data = tupleRef.getFieldData(i);
             int offset = tupleRef.getFieldStart(i);
             bbis.setByteArray(data, offset);
@@ -65,11 +66,65 @@
         return record;
     }
 
-    public Object[] deserializeRecord(ArrayTupleBuilder tb) throws HyracksDataException {
+    public Object[] deserializeRecord(IFrameTupleAccessor left, int tIndex, ITupleReference right)
+            throws HyracksDataException {
+        byte[] data = left.getBuffer().array();
+        int tStart = left.getTupleStartOffset(tIndex) + left.getFieldSlotsLength();
+        int leftFieldCount = left.getFieldCount();
+        int fStart = tStart;
+        for (int i = 0; i < leftFieldCount; ++i) {
+            /**
+             * reset the input
+             */
+            fStart = tStart + left.getFieldStartOffset(tIndex, i);
+            bbis.setByteArray(data, fStart);
+
+            /**
+             * do deserialization
+             */
+            Object instance = recordDescriptor.getFields()[i].deserialize(di);
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest(i + " " + instance);
+            }
+            record[i] = instance;
+            if (FrameConstants.DEBUG_FRAME_IO) {
+                try {
+                    if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+                        throw new HyracksDataException("Field magic mismatch");
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        for (int i = leftFieldCount; i < record.length; ++i) {
+            byte[] rightData = right.getFieldData(i - leftFieldCount);
+            int rightOffset = right.getFieldStart(i - leftFieldCount);
+            bbis.setByteArray(rightData, rightOffset);
+
+            Object instance = recordDescriptor.getFields()[i].deserialize(di);
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest(i + " " + instance);
+            }
+            record[i] = instance;
+            if (FrameConstants.DEBUG_FRAME_IO) {
+                try {
+                    if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+                        throw new HyracksDataException("Field magic mismatch");
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        return record;
+    }
+
+    public Object[] deserializeRecord(ArrayTupleBuilder tb, ITupleReference right) throws HyracksDataException {
         byte[] data = tb.getByteArray();
         int[] offset = tb.getFieldEndOffsets();
         int start = 0;
-        for (int i = 0; i < record.length; ++i) {
+        for (int i = 0; i < offset.length; ++i) {
             /**
              * reset the input
              */
@@ -94,6 +149,26 @@
                 }
             }
         }
+        for (int i = offset.length; i < record.length; ++i) {
+            byte[] rightData = right.getFieldData(i - offset.length);
+            int rightOffset = right.getFieldStart(i - offset.length);
+            bbis.setByteArray(rightData, rightOffset);
+
+            Object instance = recordDescriptor.getFields()[i].deserialize(di);
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest(i + " " + instance);
+            }
+            record[i] = instance;
+            if (FrameConstants.DEBUG_FRAME_IO) {
+                try {
+                    if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+                        throw new HyracksDataException("Field magic mismatch");
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
         return record;
     }
 }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
new file mode 100644
index 0000000..9a30647
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+/**
+ * The buffer to hold updates.
+ * We do a batch update for the B-tree during index search and join so that
+ * avoid to open/close cursors frequently.
+ */
+public class UpdateBuffer {
+
+    private int currentInUse = 0;
+    private final int pageLimit;
+    private final List<ByteBuffer> buffers = new ArrayList<ByteBuffer>();
+    private final FrameTupleAppender appender;
+    private final IHyracksTaskContext ctx;
+    private final FrameTupleReference tuple = new FrameTupleReference();
+    private final int frameSize;
+    private IFrameTupleAccessor fta;
+
+    public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) {
+        this.appender = new FrameTupleAppender(ctx.getFrameSize());
+        ByteBuffer buffer = ctx.allocateFrame();
+        this.buffers.add(buffer);
+        this.appender.reset(buffer, true);
+        this.pageLimit = numPages;
+        this.ctx = ctx;
+        this.frameSize = ctx.getFrameSize();
+        this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
+    }
+
+    public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) {
+        //by default, the update buffer has 1000 pages
+        this(1000, ctx, fieldCount);
+    }
+
+    public void setFieldCount(int fieldCount) {
+        if (fta.getFieldCount() != fieldCount) {
+            this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
+        }
+    }
+
+    public boolean appendTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+            if (currentInUse + 1 < pageLimit) {
+                // move to the new buffer
+                currentInUse++;
+                allocate(currentInUse);
+                ByteBuffer buffer = buffers.get(currentInUse);
+                appender.reset(buffer, true);
+
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    throw new HyracksDataException("tuple cannot be appended to a new frame!");
+                }
+                return true;
+            } else {
+                return false;
+            }
+        } else {
+            return true;
+        }
+    }
+
+    public void updateBTree(ITreeIndexAccessor bta) throws HyracksDataException, IndexException {
+        // batch update
+        for (int i = 0; i <= currentInUse; i++) {
+            ByteBuffer buffer = buffers.get(i);
+            fta.reset(buffer);
+            for (int j = 0; j < fta.getTupleCount(); j++) {
+                tuple.reset(fta, j);
+                bta.update(tuple);
+            }
+        }
+
+        //cleanup the buffer
+        currentInUse = 0;
+        ByteBuffer buffer = buffers.get(0);
+        appender.reset(buffer, true);
+    }
+
+    private void allocate(int index) {
+        if (index >= buffers.size()) {
+            buffers.add(ctx.allocateFrame());
+        }
+    }
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
new file mode 100644
index 0000000..39f1361
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.dataflow.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+
+public final class UpdateBufferTupleAccessor implements IFrameTupleAccessor {
+    private final int frameSize;
+    private final int fieldCount;
+    private ByteBuffer buffer;
+
+    public UpdateBufferTupleAccessor(int frameSize, int fieldCount) {
+        this.frameSize = frameSize;
+        this.fieldCount = fieldCount;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return buffer.getInt(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);
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return getFieldCount() * 4;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return fieldCount;
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index aaa7186..31b6adc 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/pom.xml
@@ -42,6 +42,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
@@ -143,5 +144,12 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-hdfs</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
 	</dependencies>
 </project>
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
index f1b98f5..cb0e339 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
@@ -17,8 +17,9 @@
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
@@ -42,6 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs2.dataflow.FileSplitsFactory;
 import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexReader;
@@ -50,26 +52,36 @@
 
 @SuppressWarnings("rawtypes")
 public class VertexFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-    private static final Logger LOGGER = Logger.getLogger(VertexFileScanOperatorDescriptor.class.getName());
     private static final long serialVersionUID = 1L;
-    private final List<InputSplit> splits;
+    private final FileSplitsFactory splitsFactory;
     private final IConfigurationFactory confFactory;
     private final int fieldSize = 2;
+    private final String[] scheduledLocations;
+    private final boolean[] executed;
 
     /**
      * @param spec
      */
     public VertexFileScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, List<InputSplit> splits,
-            IConfigurationFactory confFactory) throws HyracksException {
+            String[] scheduledLocations, IConfigurationFactory confFactory) throws HyracksException {
         super(spec, 0, 1);
-        this.splits = splits;
+        List<FileSplit> fileSplits = new ArrayList<FileSplit>();
+        for (int i = 0; i < splits.size(); i++) {
+            fileSplits.add((FileSplit) splits.get(i));
+        }
+        this.splitsFactory = new FileSplitsFactory(fileSplits);
         this.confFactory = confFactory;
+        this.scheduledLocations = scheduledLocations;
+        this.executed = new boolean[scheduledLocations.length];
+        Arrays.fill(executed, false);
         this.recordDescriptors[0] = rd;
     }
 
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
             throws HyracksDataException {
+        final List<FileSplit> splits = splitsFactory.getSplits();
+
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             private Configuration conf = confFactory.createConfiguration();
 
@@ -78,7 +90,21 @@
                 try {
                     Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
                     writer.open();
-                    loadVertices(ctx, partition);
+                    for (int i = 0; i < scheduledLocations.length; i++) {
+                        if (scheduledLocations[i].equals(ctx.getJobletContext().getApplicationContext().getNodeId())) {
+                            /**
+                             * pick one from the FileSplit queue
+                             */
+                            synchronized (executed) {
+                                if (!executed[i]) {
+                                    executed[i] = true;
+                                } else {
+                                    continue;
+                                }
+                            }
+                            loadVertices(ctx, i);
+                        }
+                    }
                     writer.close();
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
@@ -96,7 +122,7 @@
              * @throws InterruptedException
              */
             @SuppressWarnings("unchecked")
-            private void loadVertices(final IHyracksTaskContext ctx, int partitionId) throws IOException,
+            private void loadVertices(final IHyracksTaskContext ctx, int splitId) throws IOException,
                     ClassNotFoundException, InterruptedException, InstantiationException, IllegalAccessException {
                 ByteBuffer frame = ctx.allocateFrame();
                 FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -104,14 +130,8 @@
 
                 VertexInputFormat vertexInputFormat = BspUtils.createVertexInputFormat(conf);
                 TaskAttemptContext context = new TaskAttemptContext(conf, new TaskAttemptID());
-                InputSplit split = splits.get(partition);
+                InputSplit split = splits.get(splitId);
 
-                if (split instanceof FileSplit) {
-                    FileSplit fileSplit = (FileSplit) split;
-                    LOGGER.info("read file split: " + fileSplit.getPath() + " location:" + fileSplit.getLocations()[0]
-                            + " start:" + fileSplit.getStart() + " length:" + split.getLength() + " partition:"
-                            + partition);
-                }
                 VertexReader vertexReader = vertexInputFormat.createVertexReader(split, context);
                 vertexReader.initialize(split, context);
                 Vertex readerVertex = (Vertex) BspUtils.createVertex(conf);
@@ -122,7 +142,7 @@
                  * set context
                  */
                 Context mapperContext = new Mapper().new Context(conf, new TaskAttemptID(), null, null, null, null,
-                        splits.get(partition));
+                        splits.get(splitId));
                 Vertex.setContext(mapperContext);
 
                 /**
@@ -166,5 +186,4 @@
             }
         };
     }
-
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
similarity index 89%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java
rename to pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
index 356f06c..d7cbb3a 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.std;
+package edu.uci.ics.pregelix.dataflow;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -35,14 +35,14 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 
-public class FileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+public class VertexWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     private final FileSplit[] splits;
     private final IRuntimeHookFactory preHookFactory;
     private final IRuntimeHookFactory postHookFactory;
     private final IRecordDescriptorFactory inputRdFactory;
 
-    public FileWriteOperatorDescriptor(JobSpecification spec, IRecordDescriptorFactory inputRdFactory,
+    public VertexWriteOperatorDescriptor(JobSpecification spec, IRecordDescriptorFactory inputRdFactory,
             IFileSplitProvider fileSplitProvider, IRuntimeHookFactory preHookFactory,
             IRuntimeHookFactory postHookFactory) {
         super(spec, 1, 0);
@@ -60,7 +60,6 @@
             private RecordDescriptor rd0;
             private FrameDeserializer frameDeserializer;
             private PrintWriter outputWriter;
-            private final static String separator = "|";
 
             @Override
             public void open() throws HyracksDataException {
@@ -82,10 +81,7 @@
                 frameDeserializer.reset(frame);
                 while (!frameDeserializer.done()) {
                     Object[] tuple = frameDeserializer.deserializeRecord();
-                    for (int i = 0; i < tuple.length - 1; i++) {
-                        outputWriter.print(StringSerializationUtils.toString(tuple[i]));
-                        outputWriter.print(separator);
-                    }
+                    // output the vertex
                     outputWriter.print(StringSerializationUtils.toString(tuple[tuple.length - 1]));
                     outputWriter.println();
                 }
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 43b6d17..567e220 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
@@ -119,23 +119,23 @@
             Vertex.setNumEdges(numEdges);
             giraphJobIdToSuperStep.put(giraphJobId, superStep);
             giraphJobIdToMove.put(giraphJobId, false);
-            LOGGER.info("start iteration " + Vertex.getCurrentSuperstep());
+            LOGGER.info("start iteration " + Vertex.getSuperstep());
         }
         System.gc();
     }
 
     public synchronized void endSuperStep(String giraphJobId) {
         giraphJobIdToMove.put(giraphJobId, true);
-        LOGGER.info("end iteration " + Vertex.getCurrentSuperstep());
+        LOGGER.info("end iteration " + Vertex.getSuperstep());
     }
 
     @Override
     public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
         final FileReference fRef = ioManager.createWorkspaceFile(prefix);
-        List<FileReference> files = iterationToFiles.get(Vertex.getCurrentSuperstep());
+        List<FileReference> files = iterationToFiles.get(Vertex.getSuperstep());
         if (files == null) {
             files = new ArrayList<FileReference>();
-            iterationToFiles.put(Vertex.getCurrentSuperstep(), files);
+            iterationToFiles.put(Vertex.getSuperstep(), files);
         }
         files.add(fRef);
         return fRef;
diff --git a/pregelix/pregelix-dist/pom.xml b/pregelix/pregelix-dist/pom.xml
index 4cb0387..aa3d541 100644
--- a/pregelix/pregelix-dist/pom.xml
+++ b/pregelix/pregelix-dist/pom.xml
@@ -70,7 +70,7 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>pregelix-core</artifactId>
 			<version>0.2.3-SNAPSHOT</version>
-			<scope>comile</scope>
+			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
diff --git a/pregelix/pregelix-example/data/clique/clique.txt b/pregelix/pregelix-example/data/clique/clique.txt
new file mode 100755
index 0000000..08280e3
--- /dev/null
+++ b/pregelix/pregelix-example/data/clique/clique.txt
@@ -0,0 +1,7 @@
+1 2 3 4
+2 1 3 4 5
+3 1 2 4 5
+4 1 2 3
+5 6 7
+6 5 7
+7 5 6
diff --git a/pregelix/pregelix-example/pom.xml b/pregelix/pregelix-example/pom.xml
index 6b9bc89..ce8f985 100644
--- a/pregelix/pregelix-example/pom.xml
+++ b/pregelix/pregelix-example/pom.xml
@@ -41,6 +41,7 @@
 			<plugin>
 				<groupId>org.codehaus.mojo</groupId>
 				<artifactId>appassembler-maven-plugin</artifactId>
+				<version>1.3</version>
 				<executions>
 					<execution>
 						<configuration>
@@ -76,6 +77,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
index 30e88ea..74ae455 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
@@ -126,6 +126,11 @@
         }
     }
 
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
+
     public static void main(String[] args) throws Exception {
         PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getSimpleName());
         job.setVertexClass(ConnectedComponentsVertex.class);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
index 290f90e..02e1625 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
@@ -97,7 +97,7 @@
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
-        int maxIteration = this.getContext().getConfiguration().getInt(ITERATIONS, 10);
+        int maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
         if (getSuperstep() == 1) {
             tmpVertexValue.set(1.0 / getNumVertices());
             setVertexValue(tmpVertexValue);
@@ -123,13 +123,13 @@
     /**
      * Simple VertexReader that supports {@link SimplePageRankVertex}
      */
-    public static class SimplePageRankVertexReader extends
+    public static class SimulatedPageRankVertexReader extends
             GeneratedVertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
         /** Class logger */
-        private static final Logger LOG = Logger.getLogger(SimplePageRankVertexReader.class.getName());
+        private static final Logger LOG = Logger.getLogger(SimulatedPageRankVertexReader.class.getName());
         private Map<VLongWritable, FloatWritable> edges = Maps.newHashMap();
 
-        public SimplePageRankVertexReader() {
+        public SimulatedPageRankVertexReader() {
             super();
         }
 
@@ -162,12 +162,12 @@
     /**
      * Simple VertexInputFormat that supports {@link SimplePageRankVertex}
      */
-    public static class SimplePageRankVertexInputFormat extends
+    public static class SimulatedPageRankVertexInputFormat extends
             GeneratedVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
         @Override
         public VertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> createVertexReader(
                 InputSplit split, TaskAttemptContext context) throws IOException {
-            return new SimplePageRankVertexReader();
+            return new SimulatedPageRankVertexReader();
         }
     }
 
@@ -188,6 +188,11 @@
         }
     }
 
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
+
     /**
      * Simple VertexOutputFormat that supports {@link SimplePageRankVertex}
      */
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
index 2f0ca45..daafc82 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
@@ -160,6 +160,11 @@
         }
         voteToHalt();
     }
+    
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
 
     private void signalTerminate() {
         Configuration conf = getContext().getConfiguration();
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
index a018f08..199870e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
@@ -126,6 +126,11 @@
         }
         voteToHalt();
     }
+    
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
 
     public static void main(String[] args) throws Exception {
         PregelixJob job = new PregelixJob(ShortestPathsVertex.class.getSimpleName());
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java
new file mode 100644
index 0000000..83e0a6b
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/AdjacencyListWritable.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The adjacency list contains <src, list-of-neighbors>
+ */
+public class AdjacencyListWritable implements Writable {
+
+    private VLongWritable sourceVertex = new VLongWritable();
+    private Set<VLongWritable> destinationVertexes = new TreeSet<VLongWritable>();
+
+    public AdjacencyListWritable() {
+    }
+
+    public void reset() {
+        this.destinationVertexes.clear();
+    }
+
+    public void setSource(VLongWritable source) {
+        this.sourceVertex = source;
+    }
+
+    public void addNeighbor(VLongWritable neighbor) {
+        destinationVertexes.add(neighbor);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        sourceVertex = new VLongWritable();
+        destinationVertexes.clear();
+        sourceVertex.readFields(input);
+        int numberOfNeighbors = input.readInt();
+        for (int i = 0; i < numberOfNeighbors; i++) {
+            VLongWritable neighbor = new VLongWritable();
+            neighbor.readFields(input);
+            destinationVertexes.add(neighbor);
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        sourceVertex.write(output);
+        output.writeInt(destinationVertexes.size());
+        for (VLongWritable dest : destinationVertexes) {
+            dest.write(output);
+        }
+    }
+
+    public int numberOfNeighbors() {
+        return destinationVertexes.size();
+    }
+
+    public void removeNeighbor(VLongWritable v) {
+        destinationVertexes.remove(v);
+    }
+
+    public VLongWritable getSource() {
+        return sourceVertex;
+    }
+
+    public Iterator<VLongWritable> getNeighbors() {
+        return destinationVertexes.iterator();
+    }
+
+    public void cleanNonMatch(Collection<VLongWritable> matches) {
+        destinationVertexes.retainAll(matches);
+    }
+
+    public boolean isNeighbor(VLongWritable v) {
+        return destinationVertexes.contains(v);
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
new file mode 100644
index 0000000..0e22ea1
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The representation of cliques stored in a vertex.
+ */
+public class CliquesWritable implements Writable {
+
+    private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
+    private int sizeOfClique = 0;
+
+    public CliquesWritable(List<VLongWritable> cliques, int sizeOfClique) {
+        this.cliques = cliques;
+        this.sizeOfClique = sizeOfClique;
+    }
+
+    public CliquesWritable() {
+
+    }
+
+    /**
+     * Set the size of cliques.
+     * 
+     * @param sizeOfClique
+     *            the size of each maximal clique
+     */
+    public void setCliqueSize(int sizeOfClique) {
+        this.sizeOfClique = sizeOfClique;
+    }
+
+    /**
+     * Add the clique vertexes
+     * 
+     * @param cliques
+     *            the list of vertexes -- can contain multiple cliques
+     */
+    public void addCliques(CliquesWritable cliques) {
+        this.cliques.addAll(cliques.cliques);
+    }
+
+    /**
+     * Add the clique vertexes
+     * 
+     * @param cliques
+     *            the list of vertexes -- can contain multiple cliques
+     */
+    public void addCliques(List<VLongWritable> vertexes) {
+        this.cliques.addAll(vertexes);
+    }
+
+    /**
+     * @return the size of the clique
+     */
+    public int getSizeOfClique() {
+        return sizeOfClique;
+    }
+
+    /**
+     * rese the clique
+     */
+    public void reset() {
+        this.cliques.clear();
+        this.sizeOfClique = 0;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        cliques.clear();
+        int numCliques = input.readInt();
+        if (numCliques < 0) {
+            sizeOfClique = 0;
+            return;
+        }
+        sizeOfClique = input.readInt();
+        for (int i = 0; i < numCliques; i++) {
+            for (int j = 0; j < sizeOfClique; j++) {
+                VLongWritable vid = new VLongWritable();
+                vid.readFields(input);
+                cliques.add(vid);
+            }
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (sizeOfClique <= 0) {
+            output.writeInt(-1);
+            return;
+        }
+        output.writeInt(cliques.size() / sizeOfClique);
+        output.writeInt(sizeOfClique);
+
+        for (int i = 0; i < cliques.size(); i++) {
+            cliques.get(i).write(output);
+        }
+    }
+
+    @Override
+    public String toString() {
+        if (sizeOfClique == 0)
+            return "";
+        StringBuffer sb = new StringBuffer();
+        int numCliques = cliques.size() / sizeOfClique;
+        for (int i = 0; i < numCliques; i++) {
+            for (int j = 0; j < sizeOfClique - 1; j++) {
+                sb.append(cliques.get(j));
+                sb.append(",");
+            }
+            sb.append(cliques.get(sizeOfClique - 1));
+            sb.append(";");
+        }
+        return sb.toString();
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java
new file mode 100644
index 0000000..061e9e0
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueAggregator.java
@@ -0,0 +1,65 @@
+package edu.uci.ics.pregelix.example.maximalclique;
+
+import org.apache.hadoop.io.NullWritable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The global aggregator aggregates the count of triangles
+ */
+public class MaximalCliqueAggregator
+        extends
+        GlobalAggregator<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable, CliquesWritable, CliquesWritable> {
+
+    private CliquesWritable state = new CliquesWritable();
+
+    @Override
+    public void init() {
+        state.reset();
+    }
+
+    @Override
+    public void step(Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> v)
+            throws HyracksDataException {
+        CliquesWritable cliques = v.getVertexValue();
+        updateAggregateState(cliques);
+    }
+
+    /**
+     * Update the current aggregate state
+     * 
+     * @param cliques the incoming cliques
+     */
+    private void updateAggregateState(CliquesWritable cliques) {
+        if (cliques.getSizeOfClique() > state.getSizeOfClique()) {
+            //reset the vertex state
+            state.reset();
+            state.setCliqueSize(cliques.getSizeOfClique());
+            state.addCliques(cliques);
+        } else if (cliques.getSizeOfClique() == state.getSizeOfClique()) {
+            //add the new cliques
+            state.addCliques(cliques);
+        } else {
+            return;
+        }
+    }
+
+    @Override
+    public void step(CliquesWritable partialResult) {
+        updateAggregateState(partialResult);
+    }
+
+    @Override
+    public CliquesWritable finishPartial() {
+        return state;
+    }
+
+    @Override
+    public CliquesWritable finishFinal() {
+        return state;
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
new file mode 100644
index 0000000..266feb7
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
@@ -0,0 +1,347 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.maximalclique;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex;
+
+/**
+ * The maximal clique example -- find maximal cliques in an undirected graph.
+ * The result cliques contains vertexes ordered by the vertex id ascendingly. The algorithm takes
+ * advantage of that property to do effective pruning.
+ */
+public class MaximalCliqueVertex extends Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+    private Map<VLongWritable, AdjacencyListWritable> map = new TreeMap<VLongWritable, AdjacencyListWritable>();
+    private List<VLongWritable> vertexList = new ArrayList<VLongWritable>();
+    private Map<VLongWritable, Integer> invertedMap = new TreeMap<VLongWritable, Integer>();
+    private int largestCliqueSizeSoFar = 0;
+    private List<BitSet> currentMaximalCliques = new ArrayList<BitSet>();
+    private CliquesWritable tmpValue = new CliquesWritable();
+    private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
+
+    /**
+     * Update the current maximal cliques
+     * 
+     * @param values
+     *            the received adjcency lists
+     */
+    private void updateCurrentMaximalCliques(Iterator<AdjacencyListWritable> values) {
+        map.clear();
+        vertexList.clear();
+        invertedMap.clear();
+        currentMaximalCliques.clear();
+        cliques.clear();
+        tmpValue.reset();
+
+        // build the initial sub graph
+        while (values.hasNext()) {
+            AdjacencyListWritable adj = values.next();
+            map.put(adj.getSource(), adj);
+        }
+        VLongWritable srcId = getVertexId();
+        map.put(srcId, new AdjacencyListWritable());
+
+        // build the vertex list (vertex id in ascending order) and the inverted list of vertexes
+        int i = 0;
+        for (VLongWritable v : map.keySet()) {
+            vertexList.add(v);
+            invertedMap.put(v, i++);
+        }
+
+        //clean up adjacency list --- remove vertexes who are not neighbors of key
+        for (AdjacencyListWritable adj : map.values()) {
+            adj.cleanNonMatch(vertexList);
+        }
+
+        // get the h-index of the subgraph --- which is the maximum depth to explore
+        int[] neighborCounts = new int[map.size()];
+        i = 0;
+        for (AdjacencyListWritable adj : map.values()) {
+            neighborCounts[i++] = adj.numberOfNeighbors();
+        }
+        Arrays.sort(neighborCounts);
+        int h = 0;
+        for (i = neighborCounts.length - 1; i >= 0; i--) {
+            if (h >= neighborCounts[i]) {
+                break;
+            }
+            h++;
+        }
+        if (h < largestCliqueSizeSoFar) {
+            return;
+        }
+
+        //start depth-first search
+        BitSet cliqueSoFar = new BitSet(h);
+        for (VLongWritable v : vertexList) {
+            cliqueSoFar.set(invertedMap.get(v));
+            searchClique(h, cliqueSoFar, 1, v);
+            cliqueSoFar.clear();
+        }
+
+        //output local maximal cliques
+        for (BitSet clique : currentMaximalCliques) {
+            int keyIndex = invertedMap.get(srcId);
+            clique.set(keyIndex);
+            generateClique(clique);
+            tmpValue.addCliques(cliques);
+            tmpValue.setCliqueSize(clique.cardinality());
+        }
+
+        //update the vertex state
+        setVertexValue(tmpValue);
+    }
+
+    /**
+     * Output a clique with vertex ids.
+     * 
+     * @param clique
+     *            the bitmap representation of a clique
+     */
+    private void generateClique(BitSet clique) {
+        for (int j = 0; j < clique.length();) {
+            j = clique.nextSetBit(j);
+            VLongWritable v = vertexList.get(j);
+            cliques.add(v);
+            j++;
+        }
+    }
+
+    /**
+     * find cliques using the depth-first search
+     * 
+     * @param maxDepth
+     *            the maximum search depth
+     * @param cliqueSoFar
+     *            the the cliques found so far
+     * @param depthSoFar
+     *            the current search depth
+     * @param currentSource
+     *            the vertex to be added into the clique
+     */
+    private void searchClique(int maxDepth, BitSet cliqueSoFar, int depthSoFar, VLongWritable currentSource) {
+        if (depthSoFar > maxDepth) {
+            // update maximal clique info
+            updateMaximalClique(cliqueSoFar);
+            return;
+        }
+
+        AdjacencyListWritable adj = map.get(currentSource);
+        Iterator<VLongWritable> neighbors = adj.getNeighbors();
+        ++depthSoFar;
+        while (neighbors.hasNext()) {
+            VLongWritable neighbor = neighbors.next();
+            if (!isTested(neighbor, cliqueSoFar) && isClique(neighbor, cliqueSoFar)) {
+                //snapshot the clique
+                int cliqueLength = cliqueSoFar.length();
+                // expand the clique
+                cliqueSoFar.set(invertedMap.get(neighbor));
+                searchClique(maxDepth, cliqueSoFar, depthSoFar, neighbor);
+                // back to the snapshot clique
+                cliqueSoFar.set(cliqueLength, cliqueSoFar.length(), false);
+            }
+        }
+
+        // update maximal clique info
+        updateMaximalClique(cliqueSoFar);
+    }
+
+    /**
+     * Update the maximal clique to a larger one if it exists
+     * 
+     * @param cliqueSoFar
+     *            the clique so far, in the bitmap representation
+     */
+    private void updateMaximalClique(BitSet cliqueSoFar) {
+        int cliqueSize = cliqueSoFar.cardinality();
+        if (cliqueSize > largestCliqueSizeSoFar) {
+            currentMaximalCliques.clear();
+            currentMaximalCliques.add((BitSet) cliqueSoFar.clone());
+            largestCliqueSizeSoFar = cliqueSize;
+        } else if (cliqueSize == largestCliqueSizeSoFar) {
+            currentMaximalCliques.add((BitSet) cliqueSoFar.clone());
+        } else {
+            return;
+        }
+    }
+
+    /**
+     * Should we test the vertex newVertex?
+     * 
+     * @param newVertex
+     *            the vertex to be tested
+     * @param cliqueSoFar
+     *            the current clique, in the bitmap representation
+     * @return true if new vertex has been tested
+     */
+    private boolean isTested(VLongWritable newVertex, BitSet cliqueSoFar) {
+        int index = invertedMap.get(newVertex);
+        int largestSetIndex = cliqueSoFar.length() - 1;
+        if (index > largestSetIndex) {
+            // we only return cliques with vertexes in the ascending order
+            // hence, the new vertex must be larger than the largesetSetIndex in the clique
+            return false;
+        } else {
+            // otherwise, we think the vertex is "tested"
+            return true;
+        }
+    }
+
+    /**
+     * Will adding the newVertex yield a bigger clique?
+     * 
+     * @param newVertex
+     *            the new vertex id
+     * @param cliqueSoFar
+     *            the bitmap representation of the clique
+     * @return true if adding the new vertex yelds a bigger clique
+     */
+    private boolean isClique(VLongWritable newVertex, BitSet cliqueSoFar) {
+        AdjacencyListWritable adj = map.get(newVertex);
+        // check whether each existing vertex is in the neighbor set of newVertex
+        for (int i = 0; i < cliqueSoFar.length();) {
+            i = cliqueSoFar.nextSetBit(i);
+            VLongWritable v = vertexList.get(i);
+            if (!adj.isNeighbor(v)) {
+                return false;
+            }
+            i++;
+        }
+        return true;
+    }
+
+    /**
+     * For superstep 1, send outgoing mesages.
+     * For superstep 2, calculate maximal cliques.
+     * otherwise, vote to halt.
+     */
+    @Override
+    public void compute(Iterator<AdjacencyListWritable> msgIterator) {
+        if (getSuperstep() == 1) {
+            sortEdges();
+            sendOutgoingMsgs(getEdges());
+        } else if (getSuperstep() == 2) {
+            updateCurrentMaximalCliques(msgIterator);
+        } else {
+            voteToHalt();
+        }
+    }
+
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
+
+    private static CliquesWritable readMaximalCliqueResult(Configuration conf) {
+        try {
+            CliquesWritable result = (CliquesWritable) IterationUtils.readGlobalAggregateValue(conf,
+                    BspUtils.getJobId(conf));
+            return result;
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName());
+        job.setVertexClass(MaximalCliqueVertex.class);
+        job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+        job.setDynamicVertexValueSize(true);
+        job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+        job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        Client.run(args, job);
+        System.out.println("maximal cliques: \n" + readMaximalCliqueResult(job.getConfiguration()));
+    }
+
+    /**
+     * Send the adjacency lists
+     * 
+     * @param edges
+     *            the outgoing edges
+     */
+    private void sendOutgoingMsgs(List<Edge<VLongWritable, NullWritable>> edges) {
+        for (int i = 0; i < edges.size(); i++) {
+            if (edges.get(i).getDestVertexId().get() < getVertexId().get()) {
+                // only add emit for the vertexes whose id is smaller than the vertex id 
+                // to avoid the duplicate removal step,
+                // because all the resulting cliques will have vertexes in the ascending order.
+                AdjacencyListWritable msg = new AdjacencyListWritable();
+                msg.setSource(getVertexId());
+                for (int j = i + 1; j < edges.size(); j++) {
+                    msg.addNeighbor(edges.get(j).getDestVertexId());
+                }
+                sendMsg(edges.get(i).getDestVertexId(), msg);
+            }
+        }
+    }
+
+    /**
+     * Maximal Clique VertexWriter
+     */
+    public static class MaximalCliqueVertexWriter extends
+            TextVertexWriter<VLongWritable, CliquesWritable, NullWritable> {
+        public MaximalCliqueVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+            super(lineRecordWriter);
+        }
+
+        @Override
+        public void writeVertex(Vertex<VLongWritable, CliquesWritable, NullWritable, ?> vertex) throws IOException,
+                InterruptedException {
+            getRecordWriter().write(new Text(vertex.getVertexId().toString()),
+                    new Text(vertex.getVertexValue().toString()));
+        }
+    }
+
+    /**
+     * output format for maximal clique
+     */
+    public static class MaximalCliqueVertexOutputFormat extends
+            TextVertexOutputFormat<VLongWritable, CliquesWritable, NullWritable> {
+
+        @Override
+        public VertexWriter<VLongWritable, CliquesWritable, NullWritable> createVertexWriter(TaskAttemptContext context)
+                throws IOException, InterruptedException {
+            RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+            return new MaximalCliqueVertexWriter(recordWriter);
+        }
+
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java
new file mode 100644
index 0000000..ec7b32c
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/TextMaximalCliqueInputFormat.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.example.maximalclique;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat.TextVertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class TextMaximalCliqueInputFormat extends
+        TextVertexInputFormat<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+    @Override
+    public VertexReader<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> createVertexReader(
+            InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextMaximalCliqueGraphReader(textInputFormat.createRecordReader(split, context));
+    }
+}
+
+@SuppressWarnings("rawtypes")
+class TextMaximalCliqueGraphReader extends
+        TextVertexReader<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
+
+    private final static String separator = " ";
+    private Vertex vertex;
+    private VLongWritable vertexId = new VLongWritable();
+    private List<VLongWritable> pool = new ArrayList<VLongWritable>();
+    private int used = 0;
+
+    public TextMaximalCliqueGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+        super(lineRecordReader);
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+        return getRecordReader().nextKeyValue();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> getCurrentVertex()
+            throws IOException, InterruptedException {
+        used = 0;
+        if (vertex == null)
+            vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+        vertex.getMsgList().clear();
+        vertex.getEdges().clear();
+
+        vertex.reset();
+        Text line = getRecordReader().getCurrentValue();
+        String[] fields = line.toString().split(separator);
+
+        if (fields.length > 0) {
+            /**
+             * set the src vertex id
+             */
+            long src = Long.parseLong(fields[0]);
+            vertexId.set(src);
+            vertex.setVertexId(vertexId);
+            long dest = -1L;
+
+            /**
+             * set up edges
+             */
+            for (int i = 1; i < fields.length; i++) {
+                dest = Long.parseLong(fields[i]);
+                VLongWritable destId = allocate();
+                destId.set(dest);
+                vertex.addEdge(destId, null);
+            }
+        }
+        return vertex;
+    }
+
+    private VLongWritable allocate() {
+        if (used >= pool.size()) {
+            VLongWritable value = new VLongWritable();
+            pool.add(value);
+            used++;
+            return value;
+        } else {
+            VLongWritable value = pool.get(used);
+            used++;
+            return value;
+        }
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java
new file mode 100644
index 0000000..bb399ff
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TextTriangleCountingInputFormat.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.trianglecounting;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat.TextVertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class TextTriangleCountingInputFormat extends
+        TextVertexInputFormat<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+    @Override
+    public VertexReader<VLongWritable, VLongWritable, VLongWritable, VLongWritable> createVertexReader(
+            InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextPageRankGraphReader(textInputFormat.createRecordReader(split, context));
+    }
+}
+
+@SuppressWarnings("rawtypes")
+class TextPageRankGraphReader extends TextVertexReader<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+    private final static String separator = " ";
+    private Vertex vertex;
+    private VLongWritable vertexId = new VLongWritable();
+    private List<VLongWritable> pool = new ArrayList<VLongWritable>();
+    private int used = 0;
+
+    public TextPageRankGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+        super(lineRecordReader);
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+        return getRecordReader().nextKeyValue();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> getCurrentVertex() throws IOException,
+            InterruptedException {
+        used = 0;
+        if (vertex == null)
+            vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+        vertex.getMsgList().clear();
+        vertex.getEdges().clear();
+
+        vertex.reset();
+        Text line = getRecordReader().getCurrentValue();
+        String[] fields = line.toString().split(separator);
+
+        if (fields.length > 0) {
+            /**
+             * set the src vertex id
+             */
+            long src = Long.parseLong(fields[0]);
+            vertexId.set(src);
+            vertex.setVertexId(vertexId);
+            long dest = -1L;
+
+            /**
+             * set up edges
+             */
+            for (int i = 1; i < fields.length; i++) {
+                dest = Long.parseLong(fields[i]);
+                VLongWritable destId = allocate();
+                destId.set(dest);
+                vertex.addEdge(destId, null);
+            }
+        }
+        // vertex.sortEdges();
+        return vertex;
+    }
+
+    private VLongWritable allocate() {
+        if (used >= pool.size()) {
+            VLongWritable value = new VLongWritable();
+            pool.add(value);
+            used++;
+            return value;
+        } else {
+            VLongWritable value = pool.get(used);
+            used++;
+            return value;
+        }
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java
new file mode 100644
index 0000000..67b028d
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingAggregator.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.pregelix.example.trianglecounting;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The global aggregator aggregates the count of triangles
+ */
+public class TriangleCountingAggregator extends
+        GlobalAggregator<VLongWritable, VLongWritable, VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+    private VLongWritable state = new VLongWritable(0);
+
+    @Override
+    public void init() {
+        state.set(0);
+    }
+
+    @Override
+    public void step(Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> v) throws HyracksDataException {
+        state.set(state.get() + v.getVertexValue().get());
+    }
+
+    @Override
+    public void step(VLongWritable partialResult) {
+        state.set(state.get() + partialResult.get());
+    }
+
+    @Override
+    public VLongWritable finishPartial() {
+        return state;
+    }
+
+    @Override
+    public VLongWritable finishFinal() {
+        return state;
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
new file mode 100644
index 0000000..d3db095
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
@@ -0,0 +1,153 @@
+package edu.uci.ics.pregelix.example.trianglecounting;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * The triangle counting example -- counting the triangles in an undirected graph.
+ */
+public class TriangleCountingVertex extends Vertex<VLongWritable, VLongWritable, VLongWritable, VLongWritable> {
+
+    private VLongWritable tmpValue = new VLongWritable(0);
+    private long triangleCount = 0;
+    private Edge<VLongWritable, VLongWritable> candidateEdge = new Edge<VLongWritable, VLongWritable>(
+            new VLongWritable(0), new VLongWritable(0));
+    private EdgeComparator edgeComparator = new EdgeComparator();
+
+    @Override
+    public void compute(Iterator<VLongWritable> msgIterator) {
+        // transforms the edge list into a set to facilitate lookup
+        if (getSuperstep() == 1) {
+            // sorting edges could be avoid if the dataset already has that property
+            sortEdges();
+            List<Edge<VLongWritable, VLongWritable>> edges = this.getEdges();
+            int numEdges = edges.size();
+
+            //decoding longs
+            long src = getVertexId().get();
+            long[] dests = new long[numEdges];
+            for (int i = 0; i < numEdges; i++) {
+                dests[i] = edges.get(i).getDestVertexId().get();
+            }
+
+            //send messages -- take advantage of that each discovered 
+            //triangle should have vertexes ordered by vertex id
+            for (int i = 0; i < numEdges; i++) {
+                if (dests[i] < src) {
+                    for (int j = i + 1; j < numEdges; j++) {
+                        //send messages -- v_j.id > v_i.id -- guaranteed by sortEdge()
+                        if (dests[j] > src) {
+                            sendMsg(edges.get(i).getDestVertexId(), edges.get(j).getDestVertexId());
+                        }
+                    }
+                }
+            }
+        }
+        if (getSuperstep() >= 2) {
+            triangleCount = 0;
+            List<Edge<VLongWritable, VLongWritable>> edges = this.getEdges();
+            while (msgIterator.hasNext()) {
+                VLongWritable msg = msgIterator.next();
+                candidateEdge.setDestVertexId(msg);
+                if (Collections.binarySearch(edges, candidateEdge, edgeComparator) >= 0) {
+                    // if the msg value is a dest from this vertex
+                    triangleCount++;
+                }
+            }
+
+            // set vertex value
+            tmpValue.set(triangleCount);
+            setVertexValue(tmpValue);
+            voteToHalt();
+        }
+    }
+
+    /**
+     * Triangle Counting VertexWriter
+     */
+    public static class TriangleCountingVertexWriter extends
+            TextVertexWriter<VLongWritable, VLongWritable, VLongWritable> {
+        public TriangleCountingVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+            super(lineRecordWriter);
+        }
+
+        @Override
+        public void writeVertex(Vertex<VLongWritable, VLongWritable, VLongWritable, ?> vertex) throws IOException,
+                InterruptedException {
+            getRecordWriter().write(new Text(vertex.getVertexId().toString()),
+                    new Text(vertex.getVertexValue().toString()));
+        }
+    }
+
+    @Override
+    public String toString() {
+        return getVertexId() + " " + getVertexValue();
+    }
+
+    /**
+     * output format for triangle counting
+     */
+    public static class TriangleCountingVertexOutputFormat extends
+            TextVertexOutputFormat<VLongWritable, VLongWritable, VLongWritable> {
+
+        @Override
+        public VertexWriter<VLongWritable, VLongWritable, VLongWritable> createVertexWriter(TaskAttemptContext context)
+                throws IOException, InterruptedException {
+            RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+            return new TriangleCountingVertexWriter(recordWriter);
+        }
+
+    }
+
+    private static long readTriangleCountingResult(Configuration conf) {
+        try {
+            VLongWritable count = (VLongWritable) IterationUtils
+                    .readGlobalAggregateValue(conf, BspUtils.getJobId(conf));
+            return count.get();
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName());
+        job.setVertexClass(TriangleCountingVertex.class);
+        job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
+        job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+        job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        Client.run(args, job);
+        System.out.println("triangle count: " + readTriangleCountingResult(job.getConfiguration()));
+    }
+}
+
+/**
+ * The comparator for Edge<VLongWritable, VLongWritable>.
+ */
+class EdgeComparator implements Comparator<Edge<VLongWritable, VLongWritable>> {
+
+    @Override
+    public int compare(Edge<VLongWritable, VLongWritable> left, Edge<VLongWritable, VLongWritable> right) {
+        long leftValue = left.getDestVertexId().get();
+        long rightValue = right.getDestVertexId().get();
+        return leftValue > rightValue ? 1 : (leftValue < rightValue ? -1 : 0);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
similarity index 98%
rename from pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java
rename to pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
index 68b7cca..d8f704e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexAggregator.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexAggregator.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.example;
+package edu.uci.ics.pregelix.example.utils;
 
 import java.io.IOException;
 import java.util.Iterator;
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
similarity index 98%
rename from pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java
rename to pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
index 1dd6922..8421088 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/VertexSorter.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/VertexSorter.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.example;
+package edu.uci.ics.pregelix.example.utils;
 
 import java.io.IOException;
 import java.util.Iterator;
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
index 7787347..37f03a5 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
@@ -37,7 +37,7 @@
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
 import edu.uci.ics.pregelix.example.PageRankVertex;
-import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexInputFormat;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimulatedPageRankVertexInputFormat;
 import edu.uci.ics.pregelix.example.util.TestUtils;
 
 @SuppressWarnings("deprecation")
@@ -65,7 +65,7 @@
     public DataLoadTest() throws Exception {
         job = new PregelixJob(GIRAPH_JOB_NAME);
         job.setVertexClass(PageRankVertex.class);
-        job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
+        job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
         job.getConfiguration().setClass(PregelixJob.VERTEX_INDEX_CLASS, LongWritable.class, WritableComparable.class);
         job.getConfiguration().setClass(PregelixJob.VERTEX_VALUE_CLASS, DoubleWritable.class, Writable.class);
         job.getConfiguration().setClass(PregelixJob.EDGE_VALUE_CLASS, FloatWritable.class, Writable.class);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index c0b4a10..c353d84 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -26,8 +26,8 @@
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
 import edu.uci.ics.pregelix.example.PageRankVertex;
-import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexInputFormat;
 import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimulatedPageRankVertexInputFormat;
 import edu.uci.ics.pregelix.example.ReachabilityVertex;
 import edu.uci.ics.pregelix.example.ReachabilityVertex.SimpleReachibilityVertexOutputFormat;
 import edu.uci.ics.pregelix.example.ShortestPathsVertex;
@@ -35,6 +35,14 @@
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextShortestPathsInputFormat;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex;
+import edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex.MaximalCliqueVertexOutputFormat;
+import edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat;
+import edu.uci.ics.pregelix.example.trianglecounting.TextTriangleCountingInputFormat;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex;
+import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex.TriangleCountingVertexOutputFormat;
 
 public class JobGenerator {
     private static String outputBase = "src/test/resources/jobs/";
@@ -44,6 +52,9 @@
     private static String HDFS_INPUTPATH2 = "/webmapcomplex";
     private static String HDFS_OUTPUTPAH2 = "/resultcomplex";
 
+    private static String HDFS_INPUTPATH3 = "/clique";
+    private static String HDFS_OUTPUTPAH3 = "/resultclique";
+
     private static void generatePageRankJobReal(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(PageRankVertex.class);
@@ -148,7 +159,7 @@
     private static void generatePageRankJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(PageRankVertex.class);
-        job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
+        job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
@@ -157,17 +168,10 @@
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
-    private static void genPageRank() throws IOException {
-        generatePageRankJob("PageRank", outputBase + "PageRank.xml");
-        generatePageRankJobReal("PageRank", outputBase + "PageRankReal.xml");
-        generatePageRankJobRealComplex("PageRank", outputBase + "PageRankRealComplex.xml");
-        generatePageRankJobRealNoCombiner("PageRank", outputBase + "PageRankRealNoCombiner.xml");
-    }
-
     private static void generateShortestPathJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(ShortestPathsVertex.class);
-        job.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
+        job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
         job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
@@ -177,6 +181,50 @@
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
+    private static void generatePageRankJobRealDynamic(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(PageRankVertex.class);
+        job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+        job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+        job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+        job.setDynamicVertexValueSize(true);
+        FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
+        FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
+        job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+
+    private static void generateTriangleCountingJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(TriangleCountingVertex.class);
+        job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
+        job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
+        job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+        FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+
+    private static void generateMaximalCliqueJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(MaximalCliqueVertex.class);
+        job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+        job.setDynamicVertexValueSize(true);
+        job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+        job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+        FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+
+    private static void genPageRank() throws IOException {
+        generatePageRankJob("PageRank", outputBase + "PageRank.xml");
+        generatePageRankJobReal("PageRank", outputBase + "PageRankReal.xml");
+        generatePageRankJobRealDynamic("PageRank", outputBase + "PageRankRealDynamic.xml");
+        generatePageRankJobRealComplex("PageRank", outputBase + "PageRankRealComplex.xml");
+        generatePageRankJobRealNoCombiner("PageRank", outputBase + "PageRankRealNoCombiner.xml");
+    }
+
     private static void genShortestPath() throws IOException {
         generateShortestPathJob("ShortestPaths", outputBase + "ShortestPaths.xml");
         generateShortestPathJobReal("ShortestPaths", outputBase + "ShortestPathsReal.xml");
@@ -194,11 +242,20 @@
                 + "ReachibilityRealComplexNoConnectivity.xml");
     }
 
+    private static void genTriangleCounting() throws IOException {
+        generateTriangleCountingJob("Triangle Counting", outputBase + "TriangleCounting.xml");
+    }
+
+    private static void genMaximalClique() throws IOException {
+        generateMaximalCliqueJob("Maximal Clique", outputBase + "MaximalClique.xml");
+    }
+
     public static void main(String[] args) throws IOException {
         genPageRank();
         genShortestPath();
         genConnectedComponents();
         genReachibility();
+        genTriangleCounting();
+        genMaximalClique();
     }
-
 }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 89bce34..7a5bba6 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.pregelix.core.jobgen.JobGen;
 import edu.uci.ics.pregelix.core.jobgen.JobGenInnerJoin;
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoin;
+import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
@@ -44,6 +45,9 @@
     private static String HDFS_INPUTPATH2 = "/webmapcomplex";
     private static String HDFS_OUTPUTPAH2 = "/resultcomplex";
 
+    private static String HDFS_INPUTPATH3 = "/clique";
+    private static String HDFS_OUTPUTPAH3 = "/resultclique";
+
     private final PregelixJob job;
     private JobGen[] giraphJobGens;
     private final String resultFileName;
@@ -61,21 +65,24 @@
         if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH)) {
             FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
             FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
-        } else {
+        } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH2)) {
             FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
             FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
+        } else {
+            FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
+            FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         }
         job.setJobName(jobName);
         this.resultFileName = resultFile;
         this.expectedFileName = expectedFile;
-        giraphJobGens = new JobGen[3];
+        giraphJobGens = new JobGen[4];
         giraphJobGens[0] = new JobGenOuterJoin(job);
         waitawhile();
         giraphJobGens[1] = new JobGenInnerJoin(job);
         waitawhile();
         giraphJobGens[2] = new JobGenOuterJoinSort(job);
-        //waitawhile();
-        // giraphJobGens[3] = new JobGenOuterJoinSingleSort(job);
+        waitawhile();
+        giraphJobGens[3] = new JobGenOuterJoinSingleSort(job);
     }
 
     private void waitawhile() throws InterruptedException {
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
index 8a5c34b..b5abd94 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
@@ -59,6 +59,9 @@
     private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
     private static final String HDFS_PATH2 = "/webmapcomplex/";
 
+    private static final String DATA_PATH3 = "data/clique/clique.txt";
+    private static final String HDFS_PATH3 = "/clique/";
+
     private static final String HYRACKS_APP_NAME = "giraph";
     private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
     private MiniDFSCluster dfsCluster;
@@ -104,6 +107,11 @@
         dfs.mkdirs(dest);
         dfs.copyFromLocalFile(src, dest);
 
+        src = new Path(DATA_PATH3);
+        dest = new Path(HDFS_PATH3);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
         DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
         conf.writeXml(confOutput);
         confOutput.flush();
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
index 1b22b47..d89ec46 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
@@ -62,8 +62,8 @@
             if (row1.equals(row2))
                 continue;
 
-            String[] fields1 = row1.split(",");
-            String[] fields2 = row2.split(",");
+            String[] fields1 = row1.split(" ");
+            String[] fields2 = row2.split(" ");
 
             for (int j = 0; j < fields1.length; j++) {
                 if (fields1[j].equals(fields2[j])) {
@@ -71,8 +71,6 @@
                 } else if (fields1[j].indexOf('.') < 0) {
                     return false;
                 } else {
-                    fields1[j] = fields1[j].split("=")[1];
-                    fields2[j] = fields2[j].split("=")[1];
                     Double double1 = Double.parseDouble(fields1[j]);
                     Double double2 = Double.parseDouble(fields2[j]);
                     float float1 = (float) double1.doubleValue();
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
index b8efedc..45376e2 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0, edges=(1,))
-1|Vertex(id=1,value=0, edges=(1,2,))
-2|Vertex(id=2,value=0, edges=(1,2,3,))
-3|Vertex(id=3,value=0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0, edges=(11,))
-11|Vertex(id=11,value=0, edges=(11,12,))
-12|Vertex(id=12,value=0, edges=(11,12,13,))
-13|Vertex(id=13,value=0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0
+1 0
+2 0
+3 0
+4 0
+5 0
+6 0
+7 0
+8 0
+9 0
+10 0
+11 0
+12 0
+13 0
+14 0
+15 0
+16 0
+17 0
+18 0
+19 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
index ad448b2..dbc30fc 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=0, edges=(1,50,))
-1|Vertex(id=1,value=0, edges=(1,2,))
-2|Vertex(id=2,value=0, edges=(1,2,3,))
-3|Vertex(id=3,value=0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0, edges=(11,99,))
-11|Vertex(id=11,value=0, edges=(11,12,101,))
-12|Vertex(id=12,value=0, edges=(11,12,13,))
-13|Vertex(id=13,value=0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=21, edges=(22,23,24,))
-25|Vertex(id=25,value=25, edges=())
-27|Vertex(id=27,value=27, edges=())
+0 0
+1 0
+2 0
+3 0
+4 0
+5 0
+6 0
+7 0
+8 0
+9 0
+10 0
+11 0
+12 0
+13 0
+14 0
+15 0
+16 0
+17 0
+18 0
+19 0
+21 21
+25 25
+27 27
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
new file mode 100644
index 0000000..d238037
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
@@ -0,0 +1,7 @@
+1 1,2,3,4;
+2 2,3,4;
+3 
+4 
+5 
+6 
+7 
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
index f38e191..9c4d83a 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329425, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329425
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
index ab05d38..6432eda 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329426, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
index 1fc108a..2bd09e1 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=0.0072088164890121405, edges=(1,50,))
-1|Vertex(id=1,value=0.12352056961948686, edges=(1,2,))
-2|Vertex(id=2,value=0.12045670441668178, edges=(1,2,3,))
-3|Vertex(id=3,value=0.06798545786459467, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03387281259892814, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.01942600635480669, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.013661020012182747, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.0109034351563503, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.009241684574402657, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.008082028259564783, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.007208817414047232, edges=(11,99,))
-11|Vertex(id=11,value=0.07555839219845861, edges=(11,12,101,))
-12|Vertex(id=12,value=0.07249452699565352, edges=(11,12,13,))
-13|Vertex(id=13,value=0.05063539695954156, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.029644452692487822, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.018670183493927354, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.013558283213067561, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.010892790899883237, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.009240874593661061, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.008081987856433137, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0.006521739130434782, edges=(22,23,24,))
-25|Vertex(id=25,value=0.006521739130434782, edges=())
-27|Vertex(id=27,value=0.006521739130434782, edges=())
+0 0.0072088164890121405
+1 0.12352056961948686
+2 0.12045670441668178
+3 0.06798545786459467
+4 0.03387281259892814
+5 0.01942600635480669
+6 0.013661020012182747
+7 0.0109034351563503
+8 0.009241684574402657
+9 0.008082028259564783
+10 0.007208817414047232
+11 0.07555839219845861
+12 0.07249452699565352
+13 0.05063539695954156
+14 0.029644452692487822
+15 0.018670183493927354
+16 0.013558283213067561
+17 0.010892790899883237
+18 0.009240874593661061
+19 0.008081987856433137
+21 0.006521739130434782
+25 0.006521739130434782
+27 0.006521739130434782
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
new file mode 100644
index 0000000..6432eda
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
@@ -0,0 +1,20 @@
+0 0.008290140026154316
+1 0.1535152819247165
+2 0.14646839195826475
+3 0.08125113985998214
+4 0.03976979906329426
+5 0.0225041581462058
+6 0.015736276824953852
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.15351528192471647
+12 0.14646839195826472
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
index ab05d38..9a747a6 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.008290140026154316, edges=(1,))
-1|Vertex(id=1,value=0.1535152819247165, edges=(1,2,))
-2|Vertex(id=2,value=0.14646839195826475, edges=(1,2,3,))
-3|Vertex(id=3,value=0.08125113985998214, edges=(1,2,3,4,))
-4|Vertex(id=4,value=0.03976979906329426, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=0.0225041581462058, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=0.015736276824953852, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=0.012542224114863661, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=0.010628239626209894, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=0.009294348455354817, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=0.008290140026154316, edges=(11,))
-11|Vertex(id=11,value=0.15351528192471647, edges=(11,12,))
-12|Vertex(id=12,value=0.14646839195826472, edges=(11,12,13,))
-13|Vertex(id=13,value=0.08125113985998214, edges=(11,12,13,14,))
-14|Vertex(id=14,value=0.03976979906329426, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=0.0225041581462058, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=0.015736276824953852, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=0.012542224114863661, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=0.010628239626209894, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=0.009294348455354817, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.008290140026154316
+1 0.15351528192471647
+2 0.14646839195826475
+3 0.08125113985998211
+4 0.03976979906329425
+5 0.0225041581462058
+6 0.01573627682495385
+7 0.012542224114863661
+8 0.010628239626209894
+9 0.009294348455354817
+10 0.008290140026154316
+11 0.1535152819247165
+12 0.14646839195826475
+13 0.08125113985998214
+14 0.03976979906329426
+15 0.0225041581462058
+16 0.015736276824953852
+17 0.012542224114863661
+18 0.010628239626209894
+19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
index 74113a8..a1dfc0f 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=2, edges=(1,50,))
-1|Vertex(id=1,value=3, edges=(1,2,))
-2|Vertex(id=2,value=1, edges=(1,2,3,))
-3|Vertex(id=3,value=1, edges=(1,2,3,4,))
-4|Vertex(id=4,value=1, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=1, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=1, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=1, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=3, edges=(11,99,))
-11|Vertex(id=11,value=2, edges=(11,12,101,))
-12|Vertex(id=12,value=2, edges=(11,12,13,))
-13|Vertex(id=13,value=2, edges=(11,12,13,14,))
-14|Vertex(id=14,value=2, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=2, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=2, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=2, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=2, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=2, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0, edges=(22,23,24,))
-25|Vertex(id=25,value=0, edges=())
-27|Vertex(id=27,value=0, edges=())
+0 2
+1 3
+2 1
+3 1
+4 1
+5 1
+6 1
+7 1
+8 1
+9 1
+10 3
+11 2
+12 2
+13 2
+14 2
+15 2
+16 2
+17 2
+18 2
+19 2
+21 0
+25 0
+27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
index ea0edc2..1693fb2 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
@@ -1,23 +1,23 @@
-0|Vertex(id=0,value=1, edges=(1,50,))
-1|Vertex(id=1,value=1, edges=(1,2,))
-2|Vertex(id=2,value=1, edges=(1,2,3,))
-3|Vertex(id=3,value=1, edges=(1,2,3,4,))
-4|Vertex(id=4,value=1, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=1, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=1, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=1, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=1, edges=(11,99,))
-11|Vertex(id=11,value=1, edges=(11,12,101,))
-12|Vertex(id=12,value=1, edges=(11,12,13,))
-13|Vertex(id=13,value=1, edges=(11,12,13,14,))
-14|Vertex(id=14,value=1, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=1, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=1, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=1, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=1, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=1, edges=(0,11,12,13,14,15,16,17,18,19,))
-21|Vertex(id=21,value=0, edges=(22,23,24,))
-25|Vertex(id=25,value=2, edges=())
-27|Vertex(id=27,value=0, edges=())
+0 1
+1 1
+2 1
+3 1
+4 1
+5 1
+6 1
+7 1
+8 1
+9 1
+10 1
+11 1
+12 1
+13 1
+14 1
+15 1
+16 1
+17 1
+18 1
+19 1
+21 0
+25 2
+27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
index 7bb0ca3..46d1c73 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.0, edges=(1,))
-1|Vertex(id=1,value=0.0, edges=(1,2,))
-2|Vertex(id=2,value=100.0, edges=(1,2,3,))
-3|Vertex(id=3,value=300.0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=600.0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=1000.0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=1500.0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=2100.0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=2800.0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=3600.0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=4500.0, edges=(11,))
-11|Vertex(id=11,value=5500.0, edges=(11,12,))
-12|Vertex(id=12,value=6600.0, edges=(11,12,13,))
-13|Vertex(id=13,value=7800.0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=9100.0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=10500.0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=12000.0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=13600.0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=15300.0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=17100.0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.0
+1 0.0
+2 100.0
+3 300.0
+4 600.0
+5 1000.0
+6 1500.0
+7 2100.0
+8 2800.0
+9 3600.0
+10 4500.0
+11 5500.0
+12 6600.0
+13 7800.0
+14 9100.0
+15 10500.0
+16 12000.0
+17 13600.0
+18 15300.0
+19 17100.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
index f2c31a6..b42462f 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
@@ -1,20 +1,20 @@
-0|Vertex(id=0,value=0.0, edges=(1,))
-1|Vertex(id=1,value=1.0, edges=(1,2,))
-2|Vertex(id=2,value=2.0, edges=(1,2,3,))
-3|Vertex(id=3,value=3.0, edges=(1,2,3,4,))
-4|Vertex(id=4,value=4.0, edges=(1,2,3,4,5,))
-5|Vertex(id=5,value=5.0, edges=(1,2,3,4,5,6,))
-6|Vertex(id=6,value=6.0, edges=(1,2,3,4,5,6,7,))
-7|Vertex(id=7,value=7.0, edges=(1,2,3,4,5,6,7,8,))
-8|Vertex(id=8,value=8.0, edges=(1,2,3,4,5,6,7,8,9,))
-9|Vertex(id=9,value=9.0, edges=(1,2,3,4,5,6,7,8,9,10,))
-10|Vertex(id=10,value=10.0, edges=(11,))
-11|Vertex(id=11,value=11.0, edges=(11,12,))
-12|Vertex(id=12,value=12.0, edges=(11,12,13,))
-13|Vertex(id=13,value=13.0, edges=(11,12,13,14,))
-14|Vertex(id=14,value=14.0, edges=(11,12,13,14,15,))
-15|Vertex(id=15,value=15.0, edges=(11,12,13,14,15,16,))
-16|Vertex(id=16,value=16.0, edges=(11,12,13,14,15,16,17,))
-17|Vertex(id=17,value=17.0, edges=(11,12,13,14,15,16,17,18,))
-18|Vertex(id=18,value=18.0, edges=(11,12,13,14,15,16,17,18,19,))
-19|Vertex(id=19,value=19.0, edges=(0,11,12,13,14,15,16,17,18,19,))
+0 0.0
+1 1.0
+2 2.0
+3 3.0
+4 4.0
+5 5.0
+6 6.0
+7 7.0
+8 8.0
+9 9.0
+10 10.0
+11 11.0
+12 12.0
+13 13.0
+14 14.0
+15 15.0
+16 16.0
+17 17.0
+18 18.0
+19 19.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
new file mode 100644
index 0000000..4818e13
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
@@ -0,0 +1,7 @@
+1 3
+2 2
+3 0
+4 0
+5 1
+6 0
+7 0
diff --git a/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml b/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
index 1b9a4d6..71450f1 100644
--- a/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
+++ b/pregelix/pregelix-example/src/test/resources/hadoop/conf/mapred-site.xml
@@ -18,8 +18,8 @@
       <value>20</value>
    </property>
    <property>
-      <name>mapred.min.split.size</name>
-      <value>65536</value>
+      <name>mapred.max.split.size</name>
+      <value>128</value>
    </property>
 
 </configuration>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
new file mode 100644
index 0000000..616c647
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
@@ -0,0 +1,142 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Maximal Clique</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index e425b38..744e5b0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -123,7 +123,7 @@
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexInputFormat</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
new file mode 100644
index 0000000..c1a04ae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/webmap</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/result</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>pregelix.numVertices</name><value>20</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>pregelix.combinerClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
index 3719247..9e791e2 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
@@ -124,7 +124,7 @@
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexInputFormat</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
new file mode 100644
index 0000000..ee2acc1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
@@ -0,0 +1,141 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Triangle Counting</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex$TriangleCountingVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index d12cb36..1c414ff 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -42,6 +42,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
index 105d3e2..1b8fce4 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -58,6 +59,8 @@
             private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2);
             private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1);
             private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1);
+            private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2);
+            private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1);
 
             // for writing out to message channel
             private IFrameWriter writerMsg;
@@ -82,6 +85,16 @@
             private ByteBuffer bufferGlobalAggregate;
             private GlobalAggregator aggregator;
 
+            // for writing out to insert vertex channel
+            private IFrameWriter writerInsert;
+            private FrameTupleAppender appenderInsert;
+            private ByteBuffer bufferInsert;
+
+            // for writing out to delete vertex channel
+            private IFrameWriter writerDelete;
+            private FrameTupleAppender appenderDelete;
+            private ByteBuffer bufferDelete;
+
             private Vertex vertex;
             private ResetableByteArrayOutputStream bbos = new ResetableByteArrayOutputStream();
             private DataOutput output = new DataOutputStream(bbos);
@@ -90,11 +103,13 @@
             private final List<IFrameWriter> writers = new ArrayList<IFrameWriter>();
             private final List<FrameTupleAppender> appenders = new ArrayList<FrameTupleAppender>();
             private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
+            private Configuration conf;
 
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
-                this.aggregator = BspUtils.createGlobalAggregator(confFactory.createConfiguration());
+                this.conf = confFactory.createConfiguration();
+                this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
 
                 this.writerMsg = writers[0];
@@ -114,8 +129,22 @@
                 this.appenderGlobalAggregate = new FrameTupleAppender(ctx.getFrameSize());
                 this.appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
 
-                if (writers.length > 3) {
-                    this.writerAlive = writers[3];
+                this.writerInsert = writers[3];
+                this.bufferInsert = ctx.allocateFrame();
+                this.appenderInsert = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderInsert.reset(bufferInsert, true);
+                this.writers.add(writerInsert);
+                this.appenders.add(appenderInsert);
+
+                this.writerDelete = writers[4];
+                this.bufferDelete = ctx.allocateFrame();
+                this.appenderDelete = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderDelete.reset(bufferDelete, true);
+                this.writers.add(writerDelete);
+                this.appenders.add(appenderDelete);
+
+                if (writers.length > 5) {
+                    this.writerAlive = writers[5];
                     this.bufferAlive = ctx.allocateFrame();
                     this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
                     this.appenderAlive.reset(bufferAlive, true);
@@ -125,6 +154,8 @@
                 }
 
                 tbs.add(tbMsg);
+                tbs.add(tbInsert);
+                tbs.add(tbDelete);
                 tbs.add(tbAlive);
             }
 
@@ -164,6 +195,9 @@
             @Override
             public void close() throws HyracksDataException {
                 FrameTupleUtils.flushTuplesFinal(appenderMsg, writerMsg);
+                FrameTupleUtils.flushTuplesFinal(appenderInsert, writerInsert);
+                FrameTupleUtils.flushTuplesFinal(appenderDelete, writerDelete);
+
                 if (pushAlive)
                     FrameTupleUtils.flushTuplesFinal(appenderAlive, writerAlive);
                 if (!terminate) {
@@ -177,7 +211,8 @@
             private void writeOutGlobalAggregate() throws HyracksDataException {
                 try {
                     /**
-                     * get partial aggregate result and flush to the final aggregator
+                     * get partial aggregate result and flush to the final
+                     * aggregator
                      */
                     Writable agg = aggregator.finishPartial();
                     agg.write(tbGlobalAggregate.getDataOutput());
@@ -203,15 +238,27 @@
             }
 
             @Override
-            public void update(ITupleReference tupleRef) throws HyracksDataException {
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
                 try {
                     if (vertex != null && vertex.hasUpdate()) {
-                        int fieldCount = tupleRef.getFieldCount();
-                        for (int i = 1; i < fieldCount; i++) {
-                            byte[] data = tupleRef.getFieldData(i);
-                            int offset = tupleRef.getFieldStart(i);
-                            bbos.setByteArray(data, offset);
-                            vertex.write(output);
+                        if (!BspUtils.getDynamicVertexValueSize(conf)) {
+                            //in-place update
+                            int fieldCount = tupleRef.getFieldCount();
+                            for (int i = 1; i < fieldCount; i++) {
+                                byte[] data = tupleRef.getFieldData(i);
+                                int offset = tupleRef.getFieldStart(i);
+                                bbos.setByteArray(data, offset);
+                                vertex.write(output);
+                            }
+                        } else {
+                            //write the vertex id
+                            DataOutput tbOutput = cloneUpdateTb.getDataOutput();
+                            vertex.getVertexId().write(tbOutput);
+                            cloneUpdateTb.addFieldEndOffset();
+
+                            //write the vertex value
+                            vertex.write(tbOutput);
+                            cloneUpdateTb.addFieldEndOffset();
                         }
                     }
                 } catch (IOException e) {
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
index f72b059..a4d54c8 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -58,6 +59,8 @@
             private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2);
             private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1);
             private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1);
+            private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2);
+            private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1);
 
             // for writing out to message channel
             private IFrameWriter writerMsg;
@@ -82,6 +85,16 @@
             private ByteBuffer bufferTerminate;
             private boolean terminate = true;
 
+            // for writing out to insert vertex channel
+            private IFrameWriter writerInsert;
+            private FrameTupleAppender appenderInsert;
+            private ByteBuffer bufferInsert;
+
+            // for writing out to delete vertex channel
+            private IFrameWriter writerDelete;
+            private FrameTupleAppender appenderDelete;
+            private ByteBuffer bufferDelete;
+
             // dummy empty msgList
             private MsgList msgList = new MsgList();
             private ArrayIterator msgIterator = new ArrayIterator();
@@ -93,11 +106,13 @@
             private final List<IFrameWriter> writers = new ArrayList<IFrameWriter>();
             private final List<FrameTupleAppender> appenders = new ArrayList<FrameTupleAppender>();
             private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
+            private Configuration conf;
 
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
-                this.aggregator = BspUtils.createGlobalAggregator(confFactory.createConfiguration());
+                this.conf = confFactory.createConfiguration();
+                this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
 
                 this.writerMsg = writers[0];
@@ -117,8 +132,22 @@
                 this.appenderGlobalAggregate = new FrameTupleAppender(ctx.getFrameSize());
                 this.appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
 
-                if (writers.length > 3) {
-                    this.writerAlive = writers[3];
+                this.writerInsert = writers[3];
+                this.bufferInsert = ctx.allocateFrame();
+                this.appenderInsert = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderInsert.reset(bufferInsert, true);
+                this.writers.add(writerInsert);
+                this.appenders.add(appenderInsert);
+
+                this.writerDelete = writers[4];
+                this.bufferDelete = ctx.allocateFrame();
+                this.appenderDelete = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderDelete.reset(bufferDelete, true);
+                this.writers.add(writerDelete);
+                this.appenders.add(appenderDelete);
+
+                if (writers.length > 5) {
+                    this.writerAlive = writers[5];
                     this.bufferAlive = ctx.allocateFrame();
                     this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
                     this.appenderAlive.reset(bufferAlive, true);
@@ -129,6 +158,8 @@
                 msgList.reset(msgIterator);
 
                 tbs.add(tbMsg);
+                tbs.add(tbInsert);
+                tbs.add(tbDelete);
                 tbs.add(tbAlive);
             }
 
@@ -168,13 +199,16 @@
             @Override
             public void close() throws HyracksDataException {
                 FrameTupleUtils.flushTuplesFinal(appenderMsg, writerMsg);
+                FrameTupleUtils.flushTuplesFinal(appenderInsert, writerInsert);
+                FrameTupleUtils.flushTuplesFinal(appenderDelete, writerDelete);
+
                 if (pushAlive)
                     FrameTupleUtils.flushTuplesFinal(appenderAlive, writerAlive);
                 if (!terminate) {
                     writeOutTerminationState();
                 }
-                
-                /**write out global aggregate value*/
+
+                /** write out global aggregate value */
                 writeOutGlobalAggregate();
             }
 
@@ -207,15 +241,27 @@
             }
 
             @Override
-            public void update(ITupleReference tupleRef) throws HyracksDataException {
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
                 try {
                     if (vertex != null && vertex.hasUpdate()) {
-                        int fieldCount = tupleRef.getFieldCount();
-                        for (int i = 1; i < fieldCount; i++) {
-                            byte[] data = tupleRef.getFieldData(i);
-                            int offset = tupleRef.getFieldStart(i);
-                            bbos.setByteArray(data, offset);
-                            vertex.write(output);
+                        if (!BspUtils.getDynamicVertexValueSize(conf)) {
+                            //in-place update
+                            int fieldCount = tupleRef.getFieldCount();
+                            for (int i = 1; i < fieldCount; i++) {
+                                byte[] data = tupleRef.getFieldData(i);
+                                int offset = tupleRef.getFieldStart(i);
+                                bbos.setByteArray(data, offset);
+                                vertex.write(output);
+                            }
+                        } else {
+                            //write the vertex id
+                            DataOutput tbOutput = cloneUpdateTb.getDataOutput();
+                            vertex.getVertexId().write(tbOutput);
+                            cloneUpdateTb.addFieldEndOffset();
+
+                            //write the vertex value
+                            vertex.write(tbOutput);
+                            cloneUpdateTb.addFieldEndOffset();
                         }
                     }
                 } catch (IOException e) {
@@ -224,5 +270,4 @@
             }
         };
     }
-
 }