Applied the formatter to all modified files.

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_fix_issue_96@453 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index d0ded66..ae2559b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -50,7 +50,7 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
- * Contributes the runtime operator for an unnest-map representing an inverted-index search. 
+ * Contributes the runtime operator for an unnest-map representing an inverted-index search.
  */
 public class InvertedIndexPOperator extends IndexSearchPOperator {
     public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
@@ -59,15 +59,13 @@
 
     @Override
     public PhysicalOperatorTag getOperatorTag() {
-        // TODO: Currently, I'm relying on my own version of Algebricks, not the released one.
-        // Need to add this tag in Algebricks.
         return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
     }
 
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         UnnestMapOperator unnestMapOp = (UnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
         if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -79,33 +77,31 @@
         }
         InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-        
+
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
         Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
-        
+
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(metadata,
                 context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
-                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
-                jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
-                jobGenParams.getSimilarityThreshold());
+                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(),
+                keyIndexes, jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold());
         // Contribute operator in hyracks job.
         builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);        
+        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
         ILogicalOperator srcExchange = unnestMapOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
     }
-    
+
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
-            AqlCompiledMetadataDeclarations metadata, JobGenContext context,
-            JobSpecification jobSpec, UnnestMapOperator unnestMap,
-            IOperatorSchema opSchema, boolean retainInput, String datasetName,
-            Dataset dataset, String indexName,
-            ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
-            IAlgebricksConstantValue similarityThreshold) throws AlgebricksException {
-        IAObject simThresh = ((AsterixConstantValue)similarityThreshold).getObject();
+            AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
+            UnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, String datasetName,
+            Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
+            SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold)
+            throws AlgebricksException {
+        IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = metadata.findType(itemTypeName);
         int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
@@ -140,7 +136,7 @@
             tokenComparatorFactories[i] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
             tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
         }
-        
+
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
         List<LogicalVariable> outputVars = unnestMap.getVariables();
         if (retainInput) {
@@ -148,17 +144,19 @@
             VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
         RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-        
+
         int start = outputRecDesc.getFieldCount() - numPrimaryKeys;
-        IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(outputVars, start, numPrimaryKeys, typeEnv, context);
-        ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys, typeEnv, context);
-        
-        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();        
+        IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                outputVars, start, numPrimaryKeys, typeEnv, context);
+        ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys,
+                typeEnv, context);
+
+        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadata
                 .getInvertedIndexFileSplitProviders(secondarySplitsAndConstraint.first);
-        
+
         // TODO: Here we assume there is only one search key field.
         int queryField = keyFields[0];
         // Get tokenizer and search modifier factories.
@@ -166,14 +164,13 @@
                 searchModifierType, simThresh, secondaryIndex);
         IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
                 searchModifierType, searchKeyType, secondaryIndex);
-		InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(
-				jobSpec, queryField, appContext.getStorageManagerInterface(),
-				fileSplitProviders.first, fileSplitProviders.second,
-				appContext.getIndexRegistryProvider(), tokenTypeTraits,
-				tokenComparatorFactories, invListsTypeTraits,
-				invListsComparatorFactories, new BTreeDataflowHelperFactory(),
-				queryTokenizerFactory, searchModifierFactory, outputRecDesc,
-				retainInput, NoOpOperationCallbackProvider.INSTANCE);        
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp, secondarySplitsAndConstraint.second);
+        InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(jobSpec,
+                queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
+                fileSplitProviders.second, appContext.getIndexRegistryProvider(), tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+                new BTreeDataflowHelperFactory(), queryTokenizerFactory, searchModifierFactory, outputRecDesc,
+                retainInput, NoOpOperationCallbackProvider.INSTANCE);
+        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
+                secondarySplitsAndConstraint.second);
     }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 63e32ba..a9de14f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -79,8 +79,8 @@
             AssignOperator assignOp = (AssignOperator) op2;
             ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
             if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions()
-                        .get(0).getValue();
+                ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions().get(0)
+                        .getValue();
                 fid = funcExpr.getFunctionIdentifier();
             }
         }
@@ -200,8 +200,8 @@
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
-                indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));                
-                currentTop = indexUpdate;                
+                indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
+                currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
             }
         }
@@ -222,7 +222,7 @@
             IAType secondaryKeyType = (IAType) typeEnv.getVarType(secondaryKeyVar);
             if (!isNullableType(secondaryKeyType) && !forceFilter) {
                 continue;
-            }            
+            }
             ScalarFunctionCallExpression isNullFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.IS_NULL),
                     new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
@@ -245,10 +245,10 @@
         }
         return filterExpression;
     }
-    
+
     private boolean isNullableType(IAType type) {
         if (type.getTypeTag() == ATypeTag.UNION) {
-            return ((AUnionType)type).isNullableType();
+            return ((AUnionType) type).isNullableType();
         }
         return false;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index 53fe9e9..de0ee38 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -55,7 +55,8 @@
     }
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         if (context.checkIfInDontApplySet(this, op)) {
             return false;
@@ -154,7 +155,7 @@
                 break;
             }
         }
-        
+
         return hasSecondaryIndex;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
index 44610f4..f25a671 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -103,7 +103,7 @@
 
                 return true;
             }
-            
+
             if (fid.equals(AsterixBuiltinFunctions.FEED_INGEST)) {
                 if (unnest.getPositionalVariable() != null) {
                     throw new AlgebricksException("No positional variables are allowed over datasets.");
@@ -159,7 +159,6 @@
                 return true;
             }
         }
-        
 
         return false;
     }
@@ -175,7 +174,7 @@
                 AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
         return extDataSource;
     }
-    
+
     public void addPrimaryKey(List<LogicalVariable> scanVariables, IOptimizationContext context) {
         int n = scanVariables.size();
         List<LogicalVariable> head = new ArrayList<LogicalVariable>(scanVariables.subList(0, n - 1));
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 131e7b5..b378ed7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -36,12 +36,13 @@
  * Class that embodies the commonalities between rewrite rules for access methods.
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
-    
+
     private AqlCompiledMetadataDeclarations metadata;
-    
+
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
-    
-    protected static void registerAccessMethod(IAccessMethod accessMethod, Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
+
+    protected static void registerAccessMethod(IAccessMethod accessMethod,
+            Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
         List<FunctionIdentifier> funcs = accessMethod.getOptimizableFunctions();
         for (FunctionIdentifier funcIdent : funcs) {
             List<IAccessMethod> l = accessMethods.get(funcIdent);
@@ -52,21 +53,23 @@
             l.add(accessMethod);
         }
     }
-    
+
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
     }
-    
+
     protected void setMetadataDeclarations(IOptimizationContext context) {
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
         metadata = metadataProvider.getMetadataDeclarations();
     }
-    
-    protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
+
+    protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
         // The assign may be null if there is only a filter on the primary index key.
         // Match variables from lowest assign which comes directly after the dataset scan.
-        List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1).getVariables() : subTree.dataSourceScan.getVariables();
+        List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1)
+                .getVariables() : subTree.dataSourceScan.getVariables();
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
@@ -90,13 +93,12 @@
             }
         }
     }
-    
+
     /**
      * Simply picks the first index that it finds.
      * TODO: Improve this decision process by making it more systematic.
      */
-    protected Pair<IAccessMethod, Index> chooseIndex(
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected Pair<IAccessMethod, Index> chooseIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         while (amIt.hasNext()) {
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
@@ -109,7 +111,7 @@
         }
         return null;
     }
-    
+
     /**
      * Removes irrelevant access methods candidates, based on whether the
      * expressions in the query match those in the index. For example, some
@@ -117,7 +119,6 @@
      * only require a match on a prefix of fields to be applicable. This methods
      * removes all index candidates indexExprs that are definitely not
      * applicable according to the expressions involved.
-     * 
      */
     public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx) {
         Iterator<Map.Entry<Index, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
@@ -150,7 +151,7 @@
                 if (!foundKeyField) {
                     allUsed = false;
                     break;
-                }                
+                }
             }
             // If the access method requires all exprs to be matched but they are not, remove this candidate.
             if (!allUsed && accessMethod.matchAllIndexExprs()) {
@@ -164,37 +165,39 @@
             }
         }
     }
-    
+
     /**
      * Analyzes the given selection condition, filling analyzedAMs with applicable access method types.
      * At this point we are not yet consulting the metadata whether an actual index exists or not.
      */
-    protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
         // Don't consider optimizing a disjunctive condition with an index (too complicated for now).
         if (funcIdent == AlgebricksBuiltinFunctions.OR) {
             return false;
         }
-        boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);        
+        boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
             ILogicalExpression argExpr = arg.getValue();
             if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                 continue;
             }
             AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
-            boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs); 
+            boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs);
             found = found || matchFound;
         }
         return found;
     }
-    
+
     /**
      * Finds applicable access methods for the given function expression based
      * on the function identifier, and an analysis of the function's arguments.
      * Updates the analyzedAMs accordingly.
      */
-    protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
         if (funcIdent == AlgebricksBuiltinFunctions.AND) {
             return false;
@@ -207,7 +210,7 @@
         boolean atLeastOneMatchFound = false;
         // Place holder for a new analysis context in case we need one.
         AccessMethodAnalysisContext newAnalysisCtx = new AccessMethodAnalysisContext();
-        for(IAccessMethod accessMethod : relevantAMs) {
+        for (IAccessMethod accessMethod : relevantAMs) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(accessMethod);
             // Use the current place holder.
             if (analysisCtx == null) {
@@ -226,17 +229,17 @@
         }
         return atLeastOneMatchFound;
     }
-    
+
     /**
-     * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs 
+     * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs
      * from that index to the a corresponding optimizable function expression.
      * 
      * @return true if a candidate index was added to foundIndexExprs, false
      *         otherwise
-     * @throws AlgebricksException 
+     * @throws AlgebricksException
      */
-    protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex,
-            Dataset dataset, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+    protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex, Dataset dataset,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         List<Index> indexCandidates = new ArrayList<Index>();
         // Add an index to the candidates if one of the indexed fields is fieldName.
@@ -256,7 +259,8 @@
         return true;
     }
 
-    protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+    protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         for (int optFuncExprIndex = 0; optFuncExprIndex < analysisCtx.matchedFuncExprs.size(); optFuncExprIndex++) {
             for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
                 LogicalVariable var = varList.get(varIndex);
@@ -272,7 +276,8 @@
                     // Get the fieldName corresponding to the assigned variable at varIndex
                     // from the assign operator right above the datasource scan.
                     // If the expr at varIndex is not a fieldAccess we get back null.
-                    fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1), subTree.recordType, varIndex);
+                    fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1),
+                            subTree.recordType, varIndex);
                     if (fieldName == null) {
                         continue;
                     }
@@ -292,15 +297,15 @@
             }
         }
     }
-    
+
     /**
      * Returns the field name corresponding to the assigned variable at varIndex.
      * Returns null if the expr at varIndex is not a field access function.
      */
     protected String getFieldNameOfFieldAccess(AssignOperator assign, ARecordType recordType, int varIndex) {
         // Get expression corresponding to var at varIndex.
-        AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions()
-                .get(varIndex).getValue();
+        AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions().get(varIndex)
+                .getValue();
         if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return null;
         }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index ec6bf3b..fc45063 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -11,27 +11,27 @@
  * Context for analyzing the applicability of a single access method.
  */
 public class AccessMethodAnalysisContext {
-    
+
     public List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
-    
+
     // Contains candidate indexes and a list of integers that index into matchedFuncExprs.
     // We are mapping from candidate indexes to a list of function expressions 
     // that match one of the index's expressions.
     public HashMap<Index, List<Integer>> indexExprs = new HashMap<Index, List<Integer>>();
-    
+
     // Maps from index to the dataset it is indexing.
     public HashMap<Index, Dataset> indexDatasetMap = new HashMap<Index, Dataset>();
-    
+
     public void addIndexExpr(Dataset dataset, Index index, Integer exprIndex) {
-    	List<Integer> exprs = indexExprs.get(index);
-    	if (exprs == null) {
-    	    exprs = new ArrayList<Integer>();
-    		indexExprs.put(index, exprs);
-    	}
-    	exprs.add(exprIndex);
-    	indexDatasetMap.put(index, dataset);
+        List<Integer> exprs = indexExprs.get(index);
+        if (exprs == null) {
+            exprs = new ArrayList<Integer>();
+            indexExprs.put(index, exprs);
+        }
+        exprs.add(exprIndex);
+        indexDatasetMap.put(index, dataset);
     }
-    
+
     public List<Integer> getIndexExprs(Index index) {
         return indexExprs.get(index);
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index 3f3ac02..af30163 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -23,20 +23,21 @@
     protected String datasetName;
     protected boolean retainInput;
     protected boolean requiresBroadcast;
-    
+
     private final int NUM_PARAMS = 5;
-    
+
     public AccessMethodJobGenParams() {
     }
-    
-    public AccessMethodJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput, boolean requiresBroadcast) {
+
+    public AccessMethodJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
         this.indexName = indexName;
         this.indexType = indexType;
         this.datasetName = datasetName;
         this.retainInput = retainInput;
         this.requiresBroadcast = requiresBroadcast;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(indexName)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
@@ -52,7 +53,7 @@
         retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(3));
         requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
     }
-    
+
     public String getIndexName() {
         return indexName;
     }
@@ -72,7 +73,7 @@
     public boolean getRequiresBroadcast() {
         return requiresBroadcast;
     }
-    
+
     protected void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
         Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
                 new AsterixConstantValue(new AInt32(varList.size()))));
@@ -83,7 +84,7 @@
             funcArgs.add(keyVarRef);
         }
     }
-    
+
     protected int readVarList(List<Mutable<ILogicalExpression>> funcArgs, int index, List<LogicalVariable> varList) {
         int numLowKeys = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
         if (numLowKeys > 0) {
@@ -95,7 +96,7 @@
         }
         return index + numLowKeys + 1;
     }
-    
+
     protected int getNumParams() {
         return NUM_PARAMS;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index d08ce46..cd3712d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -41,7 +41,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
- * Static helper functions for rewriting plans using indexes. 
+ * Static helper functions for rewriting plans using indexes.
  */
 public class AccessMethodUtils {
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target) {
@@ -52,41 +52,38 @@
         }
         target.add(itemType);
     }
-	
-	public static ConstantExpression createStringConstant(String str) {
+
+    public static ConstantExpression createStringConstant(String str) {
         return new ConstantExpression(new AsterixConstantValue(new AString(str)));
     }
-	
-	public static ConstantExpression createInt32Constant(int i) {
+
+    public static ConstantExpression createInt32Constant(int i) {
         return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
     }
-	
-	public static ConstantExpression createBooleanConstant(boolean b) {
+
+    public static ConstantExpression createBooleanConstant(boolean b) {
         if (b) {
             return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
         } else {
             return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
         }
     }
-	
-	public static String getStringConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((AString)obj).getStringValue();
+
+    public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AString) obj).getStringValue();
     }
-	
+
     public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((AInt32)obj).getIntegerValue();
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AInt32) obj).getIntegerValue();
     }
-    
+
     public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((ABoolean)obj).getBoolean();
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((ABoolean) obj).getBoolean();
     }
-	
+
     public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
             AccessMethodAnalysisContext analysisCtx) {
         IAlgebricksConstantValue constFilterVal = null;
@@ -112,9 +109,8 @@
         analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
         return true;
     }
-    
-    public static int getNumSecondaryKeys(Index index, ARecordType recordType)
-            throws AlgebricksException {
+
+    public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
         switch (index.getIndexType()) {
             case BTREE:
             case WORD_INVIX:
@@ -133,12 +129,12 @@
             }
         }
     }
-    
+
     /**
      * Appends the types of the fields produced by the given secondary index to dest.
      */
-    public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType,
-            Index index, boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+    public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
+            boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
         if (!primaryKeysOnly) {
             switch (index.getIndexType()) {
                 case BTREE:
@@ -169,10 +165,10 @@
             dest.add(recordType.getFieldType(partitioningKey));
         }
     }
-    
-    public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType,
-            Index index, boolean primaryKeysOnly, IOptimizationContext context,
-            List<LogicalVariable> dest) throws AlgebricksException {
+
+    public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
+            boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
+            throws AlgebricksException {
         int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         int numSecondaryKeys = getNumSecondaryKeys(index, recordType);
         int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
@@ -180,7 +176,7 @@
             dest.add(context.newVar());
         }
     }
-    
+
     public static List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(Dataset dataset, ILogicalOperator unnestMapOp) {
         int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
@@ -193,11 +189,10 @@
         }
         return primaryKeyVars;
     }
-    
-    public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset,
-            ARecordType recordType, Index index, ILogicalOperator inputOp,
-            AccessMethodJobGenParams jobGenParams, IOptimizationContext context, boolean outputPrimaryKeysOnly,
-            boolean retainInput) throws AlgebricksException {
+
+    public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
+            ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
+            boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
         jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
@@ -205,32 +200,35 @@
         List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
         List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
         // Append output variables/types generated by the secondary-index search (not forwarded from input).
-        appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context, secondaryIndexUnnestVars);
-        appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);        
+        appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context,
+                secondaryIndexUnnestVars);
+        appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
         // An index search is expressed as an unnest over an index-search function.
         IFunctionInfo secondaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(secondaryIndexSearch, secondaryIndexFuncArgs);
+        UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(
+                secondaryIndexSearch, secondaryIndexFuncArgs);
         secondaryIndexSearchFunc.setReturnsUniqueValues(true);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
         // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
-        UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(
-                secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
-        secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));        
+        UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
+                new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
+        secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
         secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
         return secondaryIndexUnnestOp;
     }
-    
-    public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset, 
-            ARecordType recordType, ILogicalOperator inputOp,
-            IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {        
+
+    public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
+            ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys,
+            boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
         List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
         // Optionally add a sort on the primary-index keys before searching the primary index.
         OrderOperator order = null;
         if (sortPrimaryKeys) {
             order = new OrderOperator();
             for (LogicalVariable pkVar : primaryKeyVars) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pkVar));
+                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(pkVar));
                 order.getOrderExpressions().add(
                         new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
             }
@@ -257,11 +255,12 @@
         appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
         // An index search is expressed as an unnest over an index-search function.
         IFunctionInfo primaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
+        AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch,
+                primaryIndexFuncArgs);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
         // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
-        UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars, new MutableObject<ILogicalExpression>(primaryIndexSearchFunc),
-                primaryIndexOutputTypes, retainInput);
+        UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
+                new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
         // Fed by the order operator or the secondaryIndexUnnestOp.
         if (sortPrimaryKeys) {
             primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
@@ -272,5 +271,5 @@
         primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
         return primaryIndexUnnestOp;
     }
-    
+
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 5a11524..414dca3 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -36,15 +36,19 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
- * Class for helping rewrite rules to choose and apply BTree indexes.  
+ * Class for helping rewrite rules to choose and apply BTree indexes.
  */
 public class BTreeAccessMethod implements IAccessMethod {
 
     // Describes whether a search predicate is an open/closed interval.
     private enum LimitType {
-        LOW_INCLUSIVE, LOW_EXCLUSIVE, HIGH_INCLUSIVE, HIGH_EXCLUSIVE, EQUAL
+        LOW_INCLUSIVE,
+        LOW_EXCLUSIVE,
+        HIGH_INCLUSIVE,
+        HIGH_EXCLUSIVE,
+        EQUAL
     }
-    
+
     // TODO: There is some redundancy here, since these are listed in AlgebricksBuiltinFunctions as well.
     private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
     static {
@@ -55,16 +59,17 @@
         funcIdents.add(AlgebricksBuiltinFunctions.GT);
         funcIdents.add(AlgebricksBuiltinFunctions.NEQ);
     }
-    
+
     public static BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
         return funcIdents;
     }
-    
+
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
     }
 
@@ -80,9 +85,9 @@
     }
 
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) 
-                    throws AlgebricksException {
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = subTree.dataset;
         ARecordType recordType = subTree.recordType;
         SelectOperator select = (SelectOperator) selectRef.getValue();
@@ -91,9 +96,9 @@
         AssignOperator assign = null;
         if (assignRef != null) {
             assign = (AssignOperator) assignRef.getValue();
-        }        
-        int numSecondaryKeys = chosenIndex.getKeyFieldNames().size();        
-        
+        }
+        int numSecondaryKeys = chosenIndex.getKeyFieldNames().size();
+
         // Info on high and low keys for the BTree search predicate.
         IAlgebricksConstantValue[] lowKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
         IAlgebricksConstantValue[] highKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
@@ -101,7 +106,7 @@
         LimitType[] highKeyLimits = new LimitType[numSecondaryKeys];
         boolean[] lowKeyInclusive = new boolean[numSecondaryKeys];
         boolean[] highKeyInclusive = new boolean[numSecondaryKeys];
-        
+
         List<Integer> exprList = analysisCtx.indexExprs.get(chosenIndex);
         List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.matchedFuncExprs;
         // List of function expressions that will be replaced by the secondary-index search.
@@ -200,10 +205,12 @@
             if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
                 return false;
             }
-            if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null && lowKeyLimits[i] == null) {
+            if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null
+                    && lowKeyLimits[i] == null) {
                 return false;
             }
-            if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null && highKeyLimits[i] == null) {
+            if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null
+                    && highKeyLimits[i] == null) {
                 return false;
             }
         }
@@ -213,7 +220,7 @@
         if (highKeyLimits[0] == null) {
             highKeyInclusive[0] = true;
         }
-        
+
         // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
         // List of variables for the assign.
         ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
@@ -221,28 +228,29 @@
         ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
         int numLowKeys = createKeyVarsAndExprs(lowKeyLimits, lowKeyConstants, keyExprList, keyVarList, context);
         int numHighKeys = createKeyVarsAndExprs(highKeyLimits, highKeyConstants, keyExprList, keyVarList, context);
-        
+
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexType.BTREE,
                 dataset.getDatasetName(), false, false);
         jobGenParams.setLowKeyInclusive(lowKeyInclusive[0]);
         jobGenParams.setHighKeyInclusive(highKeyInclusive[0]);
         jobGenParams.setLowKeyVarList(keyVarList, 0, numLowKeys);
         jobGenParams.setHighKeyVarList(keyVarList, numLowKeys, numHighKeys);
-        
+
         // Assign operator that sets the secondary-index search-key fields.
         AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
         // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
         assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
         assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
-        
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset,
-                recordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
-        
+
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+
         // Generate the rest of the upstream plan which feeds the search results into the primary index.        
         UnnestMapOperator primaryIndexUnnestOp;
         boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
         if (!isPrimaryIndex) {
-            primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType, secondaryIndexUnnestOp, context, true, false, false);
+            primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
+                    secondaryIndexUnnestOp, context, true, false, false);
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
             AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
@@ -276,19 +284,18 @@
         }
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // TODO: Implement this.
         return false;
     }
 
     private int createKeyVarsAndExprs(LimitType[] keyLimits, IAlgebricksConstantValue[] keyConstants,
             ArrayList<Mutable<ILogicalExpression>> keyExprList, ArrayList<LogicalVariable> keyVarList,
-            IOptimizationContext context) {        
+            IOptimizationContext context) {
         if (keyLimits[0] == null) {
             return 0;
         }
@@ -300,8 +307,9 @@
         }
         return numKeys;
     }
-    
-    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+
+    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs,
+            List<Mutable<ILogicalExpression>> remainingFuncExprs) {
         remainingFuncExprs.clear();
         if (replacedFuncExprs.isEmpty()) {
             return;
@@ -334,7 +342,7 @@
             }
         }
     }
-    
+
     private <T> int indexOf(T value, List<T> coll) {
         int i = 0;
         for (T member : coll) {
@@ -345,9 +353,10 @@
         }
         return -1;
     }
-    
+
     private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr) {
-        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
+        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr()
+                .getFunctionIdentifier());
         LimitType limit = null;
         switch (ck) {
             case EQ: {
@@ -380,7 +389,7 @@
         }
         return limit;
     }
-    
+
     // Returns true if there is a constant value on the left-hand side  if the given optimizable function (assuming a binary function).
     public boolean constantIsOnLhs(IOptimizableFuncExpr optFuncExpr) {
         return optFuncExpr.getFuncExpr().getArguments().get(0) == optFuncExpr.getConstantVal(0);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
index 72da5a6..c377a34 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
@@ -19,14 +19,14 @@
 
     protected List<LogicalVariable> lowKeyVarList;
     protected List<LogicalVariable> highKeyVarList;
-    
+
     protected boolean lowKeyInclusive;
     protected boolean highKeyInclusive;
-    
+
     public BTreeJobGenParams() {
         super();
     }
-    
+
     public BTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
             boolean requiresBroadcast) {
         super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
@@ -36,26 +36,26 @@
         lowKeyVarList = new ArrayList<LogicalVariable>(numKeys);
         setKeyVarList(keyVarList, lowKeyVarList, startIndex, numKeys);
     }
-    
+
     public void setHighKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
         highKeyVarList = new ArrayList<LogicalVariable>(numKeys);
         setKeyVarList(keyVarList, highKeyVarList, startIndex, numKeys);
     }
-    
+
     private void setKeyVarList(List<LogicalVariable> src, List<LogicalVariable> dest, int startIndex, int numKeys) {
         for (int i = 0; i < numKeys; i++) {
             dest.add(src.get(startIndex + i));
         }
     }
-    
+
     public void setLowKeyInclusive(boolean lowKeyInclusive) {
         this.lowKeyInclusive = lowKeyInclusive;
     }
-    
+
     public void setHighKeyInclusive(boolean highKeyInclusive) {
         this.highKeyInclusive = highKeyInclusive;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         writeVarList(lowKeyVarList, funcArgs);
@@ -63,7 +63,7 @@
         writeKeyInclusive(lowKeyInclusive, funcArgs);
         writeKeyInclusive(highKeyInclusive, funcArgs);
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
@@ -71,19 +71,19 @@
         highKeyVarList = new ArrayList<LogicalVariable>();
         int nextIndex = readVarList(funcArgs, index, lowKeyVarList);
         nextIndex = readVarList(funcArgs, nextIndex, highKeyVarList);
-        readKeyInclusives(funcArgs, nextIndex);        
+        readKeyInclusives(funcArgs, nextIndex);
     }
-    
+
     private void readKeyInclusives(List<Mutable<ILogicalExpression>> funcArgs, int index) {
         lowKeyInclusive = ((ConstantExpression) funcArgs.get(index).getValue()).getValue().isTrue();
         highKeyInclusive = ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue().isTrue();
     }
-    
+
     private void writeKeyInclusive(boolean keyInclusive, List<Mutable<ILogicalExpression>> funcArgs) {
         ILogicalExpression keyExpr = keyInclusive ? ConstantExpression.TRUE : ConstantExpression.FALSE;
         funcArgs.add(new MutableObject<ILogicalExpression>(keyExpr));
     }
-    
+
     public List<LogicalVariable> getLowKeyVarList() {
         return lowKeyVarList;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
index 0f3dd29..eadbc69 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
@@ -19,13 +19,13 @@
  * with a given index.
  */
 public interface IAccessMethod {
-    
+
     /**
      * @return A list of function identifiers that are optimizable by this
      *         access method.
      */
     public List<FunctionIdentifier> getOptimizableFunctions();
-    
+
     /**
      * Analyzes the arguments of a given optimizable funcExpr to see if this
      * access method is applicable (e.g., one arg is a constant and one is a
@@ -37,8 +37,9 @@
      * @return true if funcExpr is optimizable by this access method, false
      *         otherwise
      */
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx);
-    
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx);
+
     /**
      * Indicates whether all index expressions must be matched in order for this
      * index to be applicable.
@@ -54,21 +55,21 @@
      * @return boolean
      */
     public boolean matchPrefixIndexExprs();
-    
+
     /**
      * Applies the plan transformation to use chosenIndex to optimize a selection query.
      */
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,            
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException;
-    
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException;
+
     /**
      * Applies the plan transformation to use chosenIndex to optimize a join query.
      */
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException;
-    
+    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException;
+
     /**
      * Analyzes expr to see whether it is optimizable by the given concrete index.
      */
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 71a35e7..e4555aa 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -47,24 +47,25 @@
  * to accept any subtree on one side, as long as the other side has a datasource scan.
  */
 public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-    
+
     protected Mutable<ILogicalOperator> joinRef = null;
     protected InnerJoinOperator join = null;
-	protected AbstractFunctionCallExpression joinCond = null;
-	protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();	
-	protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
-	
-	// Register access methods.
-	protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
-	static {
-	    registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
-	}
-	
+    protected AbstractFunctionCallExpression joinCond = null;
+    protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
+    protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
+
+    // Register access methods.
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    static {
+        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+    }
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         setMetadataDeclarations(context);
-        
-    	// Match operator pattern and initialize optimizable sub trees.
+
+        // Match operator pattern and initialize optimizable sub trees.
         if (!matchesOperatorPattern(opRef, context)) {
             return false;
         }
@@ -102,14 +103,14 @@
             fillSubTreeIndexExprs(rightSubTree, analyzedAMs);
         }
         pruneIndexCandidates(analyzedAMs);
-        
+
         // Choose index to be applied.
         Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
         if (chosenIndex == null) {
             context.addToDontApplySet(this, join);
             return false;
         }
-        
+
         // Apply plan transformation using chosen index.
         AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
         boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
@@ -120,7 +121,7 @@
         context.addToDontApplySet(this, join);
         return res;
     }
-    
+
     protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         // First check that the operator is a join and its condition is a function call.
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 66acab0..59b11fc 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -51,31 +51,32 @@
  * 
  */
 public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-    
-	// Operators representing the patterns to be matched:
+
+    // Operators representing the patterns to be matched:
     // These ops are set in matchesPattern()
     protected Mutable<ILogicalOperator> selectRef = null;
     protected SelectOperator select = null;
-	protected AbstractFunctionCallExpression selectCond = null;
-	protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
+    protected AbstractFunctionCallExpression selectCond = null;
+    protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
 
-	// Register access methods.
-	protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
-	static {
-	    registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
-	    registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
-	    registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
-	}
-	
+    // Register access methods.
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    static {
+        registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+    }
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         setMetadataDeclarations(context);
-        
-    	// Match operator pattern and initialize operator members.
+
+        // Match operator pattern and initialize operator members.
         if (!matchesOperatorPattern(opRef, context)) {
             return false;
         }
-        
+
         // Analyze select condition.
         Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
         if (!analyzeCondition(selectCond, subTree.assigns, analyzedAMs)) {
@@ -86,28 +87,28 @@
         if (!subTree.setDatasetAndTypeMetadata((AqlMetadataProvider) context.getMetadataProvider())) {
             return false;
         }
-        
+
         fillSubTreeIndexExprs(subTree, analyzedAMs);
         pruneIndexCandidates(analyzedAMs);
-        
+
         // Choose index to be applied.
         Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
         if (chosenIndex == null) {
             context.addToDontApplySet(this, select);
             return false;
         }
-        
+
         // Apply plan transformation using chosen index.
         AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
-        boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree, 
-                chosenIndex.second, analysisCtx, context);
+        boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree, chosenIndex.second,
+                analysisCtx, context);
         if (res) {
             OperatorPropertiesUtil.typeOpRec(opRef, context);
         }
         context.addToDontApplySet(this, select);
         return res;
     }
-    
+
     protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         // First check that the operator is a select and its condition is a function call.
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 307d9d6..dbd92c0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -65,7 +65,7 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
- * Class for helping rewrite rules to choose and apply inverted indexes.  
+ * Class for helping rewrite rules to choose and apply inverted indexes.
  */
 public class InvertedIndexAccessMethod implements IAccessMethod {
 
@@ -76,15 +76,15 @@
         EDIT_DISTANCE,
         INVALID
     }
-    
+
     private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
     static {
-        funcIdents.add(AsterixBuiltinFunctions.CONTAINS);        
+        funcIdents.add(AsterixBuiltinFunctions.CONTAINS);
         // For matching similarity-check functions. For example, similarity-jaccard-check returns a list of two items,
         // and the select condition will get the first list-item and check whether it evaluates to true. 
         funcIdents.add(AsterixBuiltinFunctions.GET_ITEM);
     }
-    
+
     // These function identifiers are matched in this AM's analyzeFuncExprArgs(), 
     // and are not visible to the outside driver.
     private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<FunctionIdentifier>();
@@ -92,23 +92,25 @@
         secondLevelFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
         secondLevelFuncIdents.add(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK);
     }
-    
+
     public static InvertedIndexAccessMethod INSTANCE = new InvertedIndexAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
-       return funcIdents;
+        return funcIdents;
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
             return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
         }
         return analyzeGetItemFuncExpr(funcExpr, assigns, analysisCtx);
     }
-    
-    public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+
+    public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         if (funcExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GET_ITEM) {
             return false;
         }
@@ -120,8 +122,8 @@
         }
         // The first arg must be a variable or a function expr.
         // If it is a variable we must track its origin in the assigns to get the original function expr.
-        if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE &&
-                arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+        if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE
+                && arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         AbstractFunctionCallExpression matchedFuncExpr = null;
@@ -131,7 +133,7 @@
         }
         // The get-item arg is a variable. Search the assigns for its origination function.
         int matchedAssignIndex = -1;
-        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {             
+        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
             VariableReferenceExpression varRefExpr = (VariableReferenceExpression) arg1;
             // Try to find variable ref expr in all assigns.
             for (int i = 0; i < assigns.size(); i++) {
@@ -162,8 +164,10 @@
         if (!secondLevelFuncIdents.contains(matchedFuncExpr.getFunctionIdentifier())) {
             return false;
         }
-        boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
-        boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
+        boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns,
+                matchedAssignIndex, analysisCtx);
+        boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex,
+                analysisCtx);
         if (selectMatchFound || joinMatchFound) {
             return true;
         }
@@ -183,7 +187,7 @@
         ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
         ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
         // We expect arg1 and arg2 to be non-constants for a join.
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT 
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
                 || arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
             return false;
         }
@@ -195,10 +199,11 @@
         if (fieldVar2 == null) {
             return false;
         }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+                new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
         return true;
     }
-    
+
     private boolean analyzeSelectSimilarityCheckFuncExprArgs(AbstractFunctionCallExpression funcExpr,
             List<AssignOperator> assigns, int matchedAssignIndex, AccessMethodAnalysisContext analysisCtx) {
         // There should be exactly three arguments.
@@ -214,11 +219,11 @@
         // Determine whether one arg is constant, and the other is non-constant.
         ILogicalExpression constArg = null;
         ILogicalExpression nonConstArg = null;
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT 
-                && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) { 
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+                && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
             constArg = arg1;
             nonConstArg = arg2;
-        } else if(arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
+        } else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
                 && arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
             constArg = arg2;
             nonConstArg = arg1;
@@ -231,21 +236,22 @@
         if (fieldVar == null) {
             return false;
         }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar }, new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar },
+                new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
         return true;
     }
-    
-    private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg, AbstractFunctionCallExpression funcExpr,
-            List<AssignOperator> assigns, int matchedAssignIndex) {
+
+    private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg,
+            AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, int matchedAssignIndex) {
         LogicalVariable fieldVar = null;
         // Analyze nonConstArg depending on similarity function.
-        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {            
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
             AbstractFunctionCallExpression nonConstFuncExpr = funcExpr;
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 nonConstFuncExpr = (AbstractFunctionCallExpression) nonConstArg;
                 // TODO: Currently, we're only looking for word and gram tokens (non hashed).
-                if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
-                        nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+                if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+                        && nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
                     return null;
                 }
                 // Find the variable that is being tokenized.
@@ -268,8 +274,8 @@
                         }
                         AbstractFunctionCallExpression childFuncExpr = (AbstractFunctionCallExpression) childExpr;
                         // If fieldVar references the result of a tokenization, then we should remember the variable being tokenized.
-                        if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
-                                childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+                        if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+                                && childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
                             break;
                         }
                         // We expect the tokenizer's argument to be a variable, otherwise we cannot apply an index.
@@ -291,12 +297,12 @@
         }
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();                
+                fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();
             }
         }
         return fieldVar;
     }
-    
+
     @Override
     public boolean matchAllIndexExprs() {
         return true;
@@ -307,19 +313,20 @@
         return false;
     }
 
-    private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, 
-            Index chosenIndex, IOptimizableFuncExpr optFuncExpr, boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+    private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, IOptimizableFuncExpr optFuncExpr,
+            boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = indexSubTree.dataset;
         ARecordType recordType = indexSubTree.recordType;
         DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
-        
+
         InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(),
                 chosenIndex.getIndexType(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // Add function-specific args such as search modifier, and possibly a similarity threshold.
         addFunctionSpecificArgs(optFuncExpr, jobGenParams);
         // Add the type of search key from the optFuncExpr.
         addSearchKeyType(optFuncExpr, indexSubTree, context, jobGenParams);
-        
+
         // Operator that feeds the secondary-index search.
         AbstractLogicalOperator inputOp = null;
         // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
@@ -332,7 +339,7 @@
             // Add key vars and exprs to argument list.
             addKeyVarsAndExprs(optFuncExpr, keyVarList, keyExprList, context);
             // Assign operator that sets the secondary-index search-key fields.
-            inputOp = new AssignOperator(keyVarList, keyExprList);            
+            inputOp = new AssignOperator(keyVarList, keyExprList);
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
             inputOp.getInputs().add(dataSourceScan.getInputs().get(0));
             inputOp.setExecutionMode(dataSourceScan.getExecutionMode());
@@ -343,14 +350,14 @@
             inputOp = (AbstractLogicalOperator) probeSubTree.root;
         }
         jobGenParams.setKeyVarList(keyVarList);
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset,
-                recordType, chosenIndex, inputOp, jobGenParams, context, true, retainInput);
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                chosenIndex, inputOp, jobGenParams, context, true, retainInput);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
-                dataset, recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
+        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+                recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
         return primaryIndexUnnestOp;
     }
-    
+
     /**
      * Returns the variable which acts as the input search key to a secondary
      * index that optimizes optFuncExpr by replacing rewriting indexSubTree
@@ -365,23 +372,23 @@
             return optFuncExpr.getLogicalVar(0);
         }
     }
-    
+
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
         IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false, false, context);
+        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false,
+                false, context);
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.dataSourceScanRef.setValue(indexPlanRootOp);
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
         Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
         // Determine probe and index subtrees based on chosen index.
@@ -395,65 +402,68 @@
             probeSubTree = leftSubTree;
         }
         IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
-        
+
         // Clone the original join condition because we may have to modify it (and we also need the original).
         InnerJoinOperator join = (InnerJoinOperator) joinRef.getValue();
         ILogicalExpression joinCond = join.getCondition().getValue().cloneExpression();
-        
+
         // Remember original live variables to make sure our new index-based plan returns exactly those vars as well.
         List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(join, originalLiveVars);
-        
+
         // Create "panic" (non indexed) nested-loop join path if necessary.
         Mutable<ILogicalOperator> panicJoinRef = null;
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             panicJoinRef = new MutableObject<ILogicalOperator>(joinRef.getValue());
-            Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree, probeSubTree, optFuncExpr, chosenIndex, context);
+            Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree,
+                    probeSubTree, optFuncExpr, chosenIndex, context);
             probeSubTree.rootRef.setValue(newProbeRootRef.getValue());
             probeSubTree.root = newProbeRootRef.getValue();
         }
         // Create regular indexed-nested loop join path.
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex, optFuncExpr, true, true, context);        
+        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex,
+                optFuncExpr, true, true, context);
         indexSubTree.dataSourceScanRef.setValue(indexPlanRootOp);
-        
+
         // Change join into a select with the same condition.
         SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond));
         topSelect.getInputs().add(indexSubTree.rootRef);
         topSelect.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(topSelect);
-        
+
         // Add a project operator on top to guarantee that our new index-based plan returns exactly the same variables as the original plan.
         ProjectOperator projectOp = new ProjectOperator(originalLiveVars);
         projectOp.getInputs().add(new MutableObject<ILogicalOperator>(topSelect));
         projectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(projectOp);
         joinRef.setValue(projectOp);
-        
+
         // Hook up the indexed-nested loop join path with the "panic" (non indexed) nested-loop join path by putting a union all on top.
         if (panicJoinRef != null) {
             // Gather live variables from the index plan and the panic plan.
             List<LogicalVariable> indexPlanLiveVars = new ArrayList<LogicalVariable>();
-        	VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
-        	List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
-        	VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
-        	if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
-        	    throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
-        	}
-        	// Create variable mapping for union all operator.
-        	List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
-        	for (int i = 0; i < indexPlanLiveVars.size(); i++) {
-        	    varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i), panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
-        	}
-        	UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
-        	unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
-        	unionAllOp.getInputs().add(panicJoinRef);
-        	unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        	context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
-        	joinRef.setValue(unionAllOp);
+            VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
+            List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
+            if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
+                throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
+            }
+            // Create variable mapping for union all operator.
+            List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+            for (int i = 0; i < indexPlanLiveVars.size(); i++) {
+                varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i),
+                        panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
+            }
+            UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
+            unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
+            unionAllOp.getInputs().add(panicJoinRef);
+            unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
+            joinRef.setValue(unionAllOp);
         }
         return true;
     }
-    
+
     private IOptimizableFuncExpr chooseOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
         // TODO: We can probably do something smarter here.
         // Pick the first expr optimizable by this index.
@@ -462,15 +472,18 @@
         return analysisCtx.matchedFuncExprs.get(firstExprIndex);
     }
 
-    private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context) throws AlgebricksException {
+    private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
+            IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context)
+            throws AlgebricksException {
         LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
-        
+
         // We split the plan into two "branches", and add selections on each side.
         AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
         replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.root));
         replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        context.computeAndSetTypeEnvironmentForOperator(replicateOp);        
-        
+        context.computeAndSetTypeEnvironmentForOperator(replicateOp);
+
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
         IVariableTypeEnvironment topTypeEnv = context.getOutputTypeEnvironment(joinRef.getValue());
         IAType inputSearchVarType = (IAType) topTypeEnv.getVarType(inputSearchVar);
@@ -478,22 +491,22 @@
         Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
                 isFilterableSelectOpRef, isNotFilterableSelectOpRef);
-        
+
         List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(indexSubTree.root, originalLiveVars);
-        
+
         // Copy the scan subtree in indexSubTree.
         Counter counter = new Counter(context.getVarCounter());
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
-        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);        
+        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
         context.setVarCounter(counter.get());
-        
+
         List<LogicalVariable> copyLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(scanSubTree, copyLiveVars);
-        
-		// Replace the inputs of the given join op, and replace variables in its
-		// condition since we deep-copied one of the scanner subtrees which
-		// changed variables. 
+
+        // Replace the inputs of the given join op, and replace variables in its
+        // condition since we deep-copied one of the scanner subtrees which
+        // changed variables. 
         InnerJoinOperator joinOp = (InnerJoinOperator) joinRef.getValue();
         // Substitute vars in the join condition due to copying of the scanSubTree.
         List<LogicalVariable> joinCondUsedVars = new ArrayList<LogicalVariable>();
@@ -502,80 +515,99 @@
             int ix = originalLiveVars.indexOf(joinCondUsedVars.get(i));
             if (ix >= 0) {
                 joinOp.getCondition().getValue().substituteVar(originalLiveVars.get(ix), copyLiveVars.get(ix));
-            }        
-        }    
+            }
+        }
         joinOp.getInputs().clear();
         joinOp.getInputs().add(new MutableObject<ILogicalOperator>(scanSubTree));
         // Make sure that the build input (which may be materialized causing blocking) comes from 
         // the split+select, otherwise the plan will have a deadlock.
         joinOp.getInputs().add(isNotFilterableSelectOpRef);
         context.computeAndSetTypeEnvironmentForOperator(joinOp);
-        
+
         // Return the new root of the probeSubTree.
         return isFilterableSelectOpRef;
     }
-    
-    private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar, IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef, Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {        
+
+    private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar,
+            IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex,
+            IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef,
+            Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
         // Create select operator for removing tuples that are not filterable.
         // First determine the proper filter function and args based on the type of the input search var.
         ILogicalExpression isFilterableExpr = null;
         switch (inputSearchVarType.getTypeTag()) {
             case STRING: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(4);
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                        inputSearchVar)));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(chosenIndex.getGramLength())));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+                        .getConstantVal(0))));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+                        .createInt32Constant(chosenIndex.getGramLength())));
                 // TODO: Currently usePrePost is hardcoded to be true.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(true)));
-                isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE), isFilterableArgs);        
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+                        .createBooleanConstant(true)));
+                isFilterableExpr = new ScalarFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE),
+                        isFilterableArgs);
                 break;
             }
             case UNORDEREDLIST:
             case ORDEREDLIST: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                        inputSearchVar)));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
-                isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE), isFilterableArgs);        
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+                        .getConstantVal(0))));
+                isFilterableExpr = new ScalarFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE),
+                        isFilterableArgs);
                 break;
             }
             default: {
             }
         }
-        SelectOperator isFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
+        SelectOperator isFilterableSelectOp = new SelectOperator(
+                new MutableObject<ILogicalExpression>(isFilterableExpr));
         isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isFilterableSelectOp);
-        
+
         // Select operator for removing tuples that are filterable.
         List<Mutable<ILogicalExpression>> isNotFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>();
         isNotFilterableArgs.add(new MutableObject<ILogicalExpression>(isFilterableExpr));
-        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
-        SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
+        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
+        SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(
+                isNotFilterableExpr));
         isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isNotFilterableSelectOp);
-        
+
         isFilterableSelectOpRef.setValue(isFilterableSelectOp);
         isNotFilterableSelectOpRef.setValue(isNotFilterableSelectOp);
     }
-    
-    private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree, IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
+
+    private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree,
+            IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
         // If we have two variables in the optFunxExpr, then we are optimizing a join.
         IAType type = null;
         ATypeTag typeTag = null;
-        if (optFuncExpr.getNumLogicalVars() == 2) {            
+        if (optFuncExpr.getNumLogicalVars() == 2) {
             // Find the type of the variable that is going to feed into the index search.
             if (optFuncExpr.getOperatorSubTree(0) == indexSubTree) {
                 // If the index is on a dataset in subtree 0, then subtree 1 will feed.
-                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(optFuncExpr.getLogicalVar(1));
+                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(
+                        optFuncExpr.getLogicalVar(1));
             } else {
                 // If the index is on a dataset in subtree 1, then subtree 0 will feed.
-                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(optFuncExpr.getLogicalVar(0));
+                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(
+                        optFuncExpr.getLogicalVar(0));
             }
             typeTag = type.getTypeTag();
-        } else { 
+        } else {
             // We are optimizing a selection query. Add the type of the search key constant.
             AsterixConstantValue constVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
             IAObject obj = constVal.getObject();
@@ -587,7 +619,7 @@
         }
         jobGenParams.setSearchKeyType(typeTag);
     }
-    
+
     private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
             jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
@@ -605,12 +637,14 @@
         }
     }
 
-    private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList, ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context) throws AlgebricksException {
+    private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList,
+            ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context)
+            throws AlgebricksException {
         // For now we are assuming a single secondary index key.
         // Add a variable and its expr to the lists which will be passed into an assign op.
         LogicalVariable keyVar = context.newVar();
         keyVarList.add(keyVar);
-        keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));        
+        keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
         return;
     }
 
@@ -631,7 +665,7 @@
             int mergeThreshold = 0;
             // We can only optimize edit distance on strings using an ngram index.
             if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getIndexType() == IndexType.NGRAM_INVIX) {
-                AString astr = (AString) listOrStrObj;                    
+                AString astr = (AString) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
                         - edThresh.getIntegerValue() * index.getGramLength();
@@ -639,7 +673,7 @@
             // We can only optimize edit distance on lists using a word index.
             if ((listOrStrObj.getType().getTypeTag() == ATypeTag.ORDEREDLIST || listOrStrObj.getType().getTypeTag() == ATypeTag.UNORDEREDLIST)
                     && index.getIndexType() == IndexType.WORD_INVIX) {
-                IACollection alist = (IACollection) listOrStrObj;        
+                IACollection alist = (IACollection) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = alist.size() - edThresh.getIntegerValue();
             }
@@ -663,12 +697,12 @@
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 AbstractFunctionCallExpression nonConstfuncExpr = (AbstractFunctionCallExpression) nonConstArg;
                 // We can use this index if the tokenization function matches the index type.
-                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS &&
-                        index.getIndexType() == IndexType.WORD_INVIX) {
+                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS
+                        && index.getIndexType() == IndexType.WORD_INVIX) {
                     return true;
                 }
-                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS &&
-                        index.getIndexType() == IndexType.NGRAM_INVIX) {
+                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
+                        && index.getIndexType() == IndexType.NGRAM_INVIX) {
                     return true;
                 }
             }
@@ -686,7 +720,7 @@
             AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
             IAObject strObj = strConstVal.getObject();
             if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
-                AString astr = (AString) strObj;                    
+                AString astr = (AString) strObj;
                 if (astr.getStringValue().length() >= index.getGramLength()) {
                     return true;
                 }
@@ -694,10 +728,10 @@
         }
         return false;
     }
-    
+
     public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
         IAType type = keyType;
-        ATypeTag typeTag = keyType.getTypeTag();        
+        ATypeTag typeTag = keyType.getTypeTag();
         // Extract item type from list.
         if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
             AbstractCollectionType listType = (AbstractCollectionType) keyType;
@@ -707,10 +741,9 @@
             type = listType.getItemType();
         }
         // Ignore case for string types.
-        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                type, true, true);
+        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
     }
-    
+
     public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
         IAType type = keyType;
         ATypeTag typeTag = keyType.getTypeTag();
@@ -731,7 +764,7 @@
         }
         return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
     }
-    
+
     public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
             ATypeTag searchKeyType, Index index) throws AlgebricksException {
         switch (index.getIndexType()) {
@@ -749,7 +782,7 @@
             }
         }
     }
-    
+
     public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
             int gramLength) throws AlgebricksException {
         switch (indexType) {
@@ -765,7 +798,7 @@
             }
         }
     }
-    
+
     public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
             IAObject simThresh, Index index) throws AlgebricksException {
         switch (searchModifierType) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index 824ea0f..530606e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -20,44 +20,47 @@
  * and from a list of function arguments, typically of an unnest-map.
  */
 public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
-    
+
     protected SearchModifierType searchModifierType;
     protected IAlgebricksConstantValue similarityThreshold;
     protected ATypeTag searchKeyType;
     protected List<LogicalVariable> keyVarList;
     protected List<LogicalVariable> nonKeyVarList;
-    
+
     public InvertedIndexJobGenParams() {
     }
-    
-    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput, boolean requiresBroadcast) {
+
+    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
         super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
     }
-    
+
     public void setSearchModifierType(SearchModifierType searchModifierType) {
         this.searchModifierType = searchModifierType;
     }
-        
+
     public void setSimilarityThreshold(IAlgebricksConstantValue similarityThreshold) {
         this.similarityThreshold = similarityThreshold;
     }
-    
+
     public void setSearchKeyType(ATypeTag searchKeyType) {
         this.searchKeyType = searchKeyType;
     }
-    
+
     public void setKeyVarList(List<LogicalVariable> keyVarList) {
         this.keyVarList = keyVarList;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         // Write search modifier type.
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType
+                .ordinal())));
         // Write similarity threshold.
         funcArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(similarityThreshold)));
         // Write search key type.
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType
+                .ordinal())));
         // Write key var list.
         writeVarList(keyVarList, funcArgs);
         // Write non-key var list.
@@ -65,7 +68,7 @@
             writeVarList(nonKeyVarList, funcArgs);
         }
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
@@ -73,7 +76,8 @@
         int searchModifierOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
         searchModifierType = SearchModifierType.values()[searchModifierOrdinal];
         // Read similarity threshold. Concrete type depends on search modifier.
-        similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue());
+        similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue())
+                .getValue());
         // Read type of search key.
         int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + 2));
         searchKeyType = ATypeTag.values()[typeTagOrdinal];
@@ -84,23 +88,23 @@
         // We don't need to read the non-key var list.
         nonKeyVarList = null;
     }
-    
+
     public SearchModifierType getSearchModifierType() {
         return searchModifierType;
     }
-    
+
     public IAlgebricksConstantValue getSimilarityThreshold() {
         return similarityThreshold;
     }
-    
+
     public ATypeTag getSearchKeyType() {
         return searchKeyType;
     }
-    
+
     public List<LogicalVariable> getKeyVarList() {
         return keyVarList;
     }
-    
+
     public List<LogicalVariable> getNonKeyVarList() {
         return nonKeyVarList;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 69651f3..80f8cc1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -22,7 +22,7 @@
 
 /**
  * Operator subtree that matches the following patterns, and provides convenient access to its nodes:
- * (select)? <-- (assign)+ <-- (datasource scan) 
+ * (select)? <-- (assign)+ <-- (datasource scan)
  * and
  * (select)? <-- (datasource scan)
  */
@@ -35,8 +35,8 @@
     public DataSourceScanOperator dataSourceScan = null;
     // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
     public Dataset dataset = null;
-    public ARecordType recordType = null;    
-    
+    public ARecordType recordType = null;
+
     public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
         rootRef = subTreeOpRef;
         root = subTreeOpRef.getValue();
@@ -62,7 +62,7 @@
             assignRefs.add(subTreeOpRef);
             assigns.add((AssignOperator) subTreeOp);
             subTreeOpRef = subTreeOp.getInputs().get(0);
-            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();   
+            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
         } while (subTreeOp.getOperatorTag() == LogicalOperatorTag.ASSIGN);
         // Set to last valid assigns.
         subTreeOpRef = assignRefs.get(assignRefs.size() - 1);
@@ -77,11 +77,11 @@
         dataSourceScan = (DataSourceScanOperator) op3;
         return true;
     }
-    
+
     /**
      * Find the dataset corresponding to the datasource scan in the metadata.
      * Also sets recordType to be the type of that dataset.
-     */  
+     */
     public boolean setDatasetAndTypeMetadata(AqlMetadataProvider metadataProvider) throws AlgebricksException {
         if (dataSourceScan == null) {
             return false;
@@ -107,7 +107,7 @@
         recordType = (ARecordType) itemType;
         return true;
     }
-    
+
     public boolean hasDataSourceScan() {
         return dataSourceScan != null;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
index f5f94d0..dfd3ff7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
- * Class for helping rewrite rules to choose and apply RTree indexes.  
+ * Class for helping rewrite rules to choose and apply RTree indexes.
  */
 public class RTreeAccessMethod implements IAccessMethod {
 
@@ -39,19 +39,20 @@
     static {
         funcIdents.add(AsterixBuiltinFunctions.SPATIAL_INTERSECT);
     }
-    
+
     public static RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
         return funcIdents;
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
     }
-    
+
     @Override
     public boolean matchAllIndexExprs() {
         return true;
@@ -63,9 +64,9 @@
     }
 
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            Index index, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index index, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = subTree.dataset;
         ARecordType recordType = subTree.recordType;
         // TODO: We can probably do something smarter here based on selectivity or MBR area.
@@ -73,14 +74,14 @@
         List<Integer> indexExprs = analysisCtx.getIndexExprs(index);
         int firstExprIndex = indexExprs.get(0);
         IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(firstExprIndex);
-        
+
         // Get the number of dimensions corresponding to the field indexed by
         // chosenIndex.
         Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(optFuncExpr.getFieldName(0), recordType);
         IAType spatialType = keyPairType.first;
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         int numSecondaryKeys = numDimensions * 2;
-        
+
         DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
         // TODO: For now retainInput and requiresBroadcast are always false.
         RTreeJobGenParams jobGenParams = new RTreeJobGenParams(index.getIndexName(), IndexType.RTREE,
@@ -97,14 +98,16 @@
             AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CREATE_MBR));
             // Spatial object is the constant from the func expr we are optimizing.
-            createMBR.getArguments().add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
+            createMBR.getArguments().add(
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
             // The number of dimensions.
             createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                            new AInt32(numDimensions)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
+                            numDimensions)))));
             // Which part of the MBR to extract.
             createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
             // Add a variable and its expr to the lists which will be passed into an assign op.
             LogicalVariable keyVar = context.newVar();
             keyVarList.add(keyVar);
@@ -118,25 +121,24 @@
         assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
         assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
 
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset,
-                recordType, index, assignSearchKeys, jobGenParams, context, false, false);
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                index, assignSearchKeys, jobGenParams, context, false, false);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
-                dataset, recordType, secondaryIndexUnnestOp, context, true, false, false);
+        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+                recordType, secondaryIndexUnnestOp, context, true, false, false);
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            Index chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // TODO Implement this.
         return false;
     }
-    
+
     @Override
     public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         // No additional analysis required.
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
index 33f79d2..b3153f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
@@ -14,32 +14,33 @@
  * and from a list of function arguments, typically of an unnest-map.
  */
 public class RTreeJobGenParams extends AccessMethodJobGenParams {
-    
+
     protected List<LogicalVariable> keyVarList;
-    
+
     public RTreeJobGenParams() {
     }
-    
-    public RTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput, boolean requiresBroadcast) {
+
+    public RTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
         super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         writeVarList(keyVarList, funcArgs);
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
         keyVarList = new ArrayList<LogicalVariable>();
         readVarList(funcArgs, index, keyVarList);
     }
-    
+
     public void setKeyVarList(List<LogicalVariable> keyVarList) {
         this.keyVarList = keyVarList;
     }
-    
+
     public List<LogicalVariable> getKeyVarList() {
         return keyVarList;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index 3ef3160..e004a7d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -241,8 +241,8 @@
                 @SuppressWarnings("unchecked")
                 ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
                         new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AString(
-                                keyFieldName)))));
+                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                                new AString(keyFieldName)))));
                 f.substituteVar(METADATA_DUMMY_VAR, resVar);
                 exprs.add(new MutableObject<ILogicalExpression>(f));
                 LogicalVariable v = context.newVar();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 22ccdcf..d8f294d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -145,17 +145,13 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 
 /**
- * 
  * Each visit returns a pair of an operator and a variable. The variable
  * corresponds to the new column, if any, added to the tuple flow. E.g., for
  * Unnest, the column is the variable bound to the elements in the list, for
  * Subplan it is null.
- * 
  * The first argument of a visit method is the expression which is translated.
- * 
  * The second argument of a visit method is the tuple source for the current
  * subtree.
- * 
  */
 
 public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
@@ -261,15 +257,15 @@
             }
             if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
                 throw new AlgebricksException("Cannot write output to an external dataset.");
-            }            
+            }
             ARecordType itemType = (ARecordType) compiledDeclarations.findType(dataset.getItemTypeName());
             List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
             ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
             for (String partitioningKey : partitioningKeys) {
-                Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner =
-                        format.partitioningEvaluatorFactory(itemType, partitioningKey);
+                Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner = format
+                        .partitioningEvaluatorFactory(itemType, partitioningKey);
                 AbstractFunctionCallExpression f = partitioner.second.cloneExpression();
                 f.substituteVar(METADATA_DUMMY_VAR, resVar);
                 exprs.add(new MutableObject<ILogicalExpression>(f));
@@ -324,8 +320,8 @@
             returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)));
         } else {
             LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
-            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)), pVar,
-                    BuiltinType.AINT32);
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)),
+                    pVar, BuiltinType.AINT32);
         }
         returnedOp.getInputs().add(eo.second);
 
@@ -342,8 +338,8 @@
             case VARIABLE_EXPRESSION: {
                 v = context.newVar(lc.getVarExpr());
                 LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
-                returnedOp = new AssignOperator(v,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(prev)));
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(prev)));
                 returnedOp.getInputs().add(tupSource);
                 break;
             }
@@ -432,15 +428,16 @@
             f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
             f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
             f.getArguments().add(
-                    new  MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
         }
-        AssignOperator a = new AssignOperator(v, new  MutableObject<ILogicalExpression>(f));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
         a.getInputs().add(p.second);
         return new Pair<ILogicalOperator, LogicalVariable>(a, v);
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall,  Mutable<ILogicalOperator> tupSource)
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
             throws AsterixException {
         LogicalVariable v = context.newVar();
         AsterixFunction fid = fcall.getIdent();
@@ -472,11 +469,11 @@
             }
         }
 
-        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(), false);
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(),
+                false);
         AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
         FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
@@ -506,7 +503,8 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
         // TODO Auto-generated method stub
         throw new NotImplementedException();
     }
@@ -550,9 +548,10 @@
             AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
                     AsterixBuiltinFunctions.LISTIFY, flArgs);
             AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
-                    (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
             agg.getInputs().add(
-                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(gOp))));
+                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                            new MutableObject<ILogicalOperator>(gOp))));
             ILogicalPlan plan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(agg));
             gOp.getNestedPlans().add(plan);
             // Hide the variable that was part of the "with", replacing it with
@@ -586,8 +585,8 @@
                 new MutableObject<ILogicalOperator>(sp)));
 
         Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
-        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(
-                new VariableReferenceExpression(varCond)));
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
         sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
         Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
@@ -607,9 +606,9 @@
 
         LogicalVariable resV = context.newVar();
         AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(pThen.second)), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(pElse.second)));
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pThen.second)),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pElse.second)));
         AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
         a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
@@ -628,8 +627,8 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
         ArrayList<OperatorType> ops = op.getOpList();
         int nOps = ops.size();
 
@@ -686,7 +685,8 @@
                     }
                 }
             } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<ILogicalExpression>(e));
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
                 if (i == 1 && op.isBroadcastOperand(i)) {
                     BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
                     bcast.setObject(BroadcastSide.RIGHT);
@@ -715,8 +715,9 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(e, topOp);
             OrderModifier m = modifIter.next();
             OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
-            ord.getOrderExpressions().add(
-                    new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(p.first)));
+            ord.getOrderExpressions()
+                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(
+                            p.first)));
             topOp = p.second;
         }
         ord.getInputs().add(topOp);
@@ -778,7 +779,7 @@
         }
         LogicalVariable qeVar = context.newVar();
         AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
         a.getInputs().add(new MutableObject<ILogicalOperator>(s));
         return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
     }
@@ -851,8 +852,8 @@
         // Should we ever get to this method?
         LogicalVariable var = context.newVar();
         LogicalVariable oldV = context.getVar(v.getVar().getId());
-        AssignOperator a = new AssignOperator(var,
-                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldV)));
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                new VariableReferenceExpression(oldV)));
         a.getInputs().add(tupSource);
         return new Pair<ILogicalOperator, LogicalVariable>(a, var);
     }
@@ -1125,7 +1126,7 @@
         funAgg.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
         LogicalVariable varListified = context.newVar();
         AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
-                (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
         agg.getInputs().add(opRef);
         ILogicalOperator res;
         if (bProject) {
@@ -1241,7 +1242,6 @@
         return null;
     }
 
-  
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
             Mutable<ILogicalOperator> arg) throws AsterixException {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 61a0224..b028e06 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -106,9 +106,9 @@
         List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         int numSecondaryIndexes = 0;
         for (Index index : datasetIndexes) {
-        	if (index.isSecondaryIndex()) {
-        		numSecondaryIndexes++;
-        	}
+            if (index.isSecondaryIndex()) {
+                numSecondaryIndexes++;
+            }
         }
         JobSpecification[] specs;
         if (numSecondaryIndexes > 0) {
@@ -116,16 +116,17 @@
             int i = 0;
             // First, drop secondary indexes.
             for (Index index : datasetIndexes) {
-            	if (index.isSecondaryIndex()) {
-            		specs[i] = new JobSpecification();
-            		Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
-            				.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, index.getIndexName());
-            		TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
-            				storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
-            		AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
-            				idxSplitsAndConstraint.second);
-            		i++;
-            	}
+                if (index.isSecondaryIndex()) {
+                    specs[i] = new JobSpecification();
+                    Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
+                            .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
+                                    index.getIndexName());
+                    TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
+                            storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+                    AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
+                            idxSplitsAndConstraint.second);
+                    i++;
+                }
             }
         } else {
             specs = new JobSpecification[1];
@@ -153,8 +154,8 @@
         }
         ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         JobSpecification spec = new JobSpecification();
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                dataset, itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
@@ -183,8 +184,7 @@
         if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName);
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL
-                && dataset.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             throw new AsterixException("Cannot load data into dataset  (" + datasetName + ")" + "of type "
                     + dataset.getDatasetType());
         }
@@ -194,14 +194,14 @@
         IDataFormat format = metadata.getFormat();
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
 
-        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(
-                dataset, itemType, metadata.getFormat().getBinaryHashFunctionFactoryProvider());
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                dataset, itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(dataset, itemType,
+                metadata.getFormat().getBinaryHashFunctionFactoryProvider());
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
 
-        ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(
-                loadStmt.getAdapter(), loadStmt.getProperties());
+        ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
+                loadStmt.getProperties());
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider
                 .buildExternalDataScannerRuntime(spec, itemType, externalDatasetDetails, format);
         IOperatorDescriptor scanner = p.first;
@@ -274,13 +274,14 @@
         return fs.getNodeName() + ":" + fs.getLocalFile().toString();
     }
 
-    private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType, IDataFormat format) throws AlgebricksException {
+    private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType,
+            IDataFormat format) throws AlgebricksException {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         int numKeys = partitioningKeys.size();
         ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = 
-                    format.partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
+            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
+                    .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
             evalFactories[i] = evalFactoryAndType.first;
         }
         int[] outColumns = new int[numKeys];
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index 66beb76..66a5d56 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -100,8 +100,8 @@
 
         try {
             Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider.buildFeedMessengerRuntime(
-                    spec, metadata, (FeedDatasetDetails) dataset.getDatasetDetails(),
-                    metadata.getDataverseName(), datasetName, feedMessages);
+                    spec, metadata, (FeedDatasetDetails) dataset.getDatasetDetails(), metadata.getDataverseName(),
+                    datasetName, feedMessages);
             feedMessenger = p.first;
             messengerPc = p.second;
         } catch (AlgebricksException e) {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 60e3c31..65f6f5e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -152,7 +152,7 @@
         ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
         primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
-        ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();        
+        ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
         for (int i = 0; i < numPrimaryKeys; i++) {
             IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 29e0c3c..591154a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -91,7 +91,7 @@
     private MetadataNode() {
         super();
     }
-    
+
     public void initialize(AsterixAppRuntimeContext runtimeContext) {
         this.transactionProvider = runtimeContext.getTransactionProvider();
         this.indexRegistry = runtimeContext.getIndexRegistry();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
index 00d723a..7c6d9ed 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
@@ -97,7 +97,7 @@
     public boolean isSecondaryIndex() {
         return !isPrimaryIndex();
     }
-    
+
     public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
             throws AlgebricksException {
         IAType keyType = Index.keyFieldType(expr, recType);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index a391e7c..2eda4a8 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -147,7 +147,7 @@
                             .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX))
                             .getStringValue();
 
-                     datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
+                    datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
                             partitioningKey, groupName, adapter, properties, functionIdentifier, feedState);
                 }
                 break;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index 226bda5..0e2f825 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -26,7 +26,7 @@
         this.fieldTypes = fieldTypes;
         this.isOpen = isOpen;
         for (int i = 0; i < fieldNames.length; i++) {
-        	typeMap.put(fieldNames[i], i);
+            typeMap.put(fieldNames[i], i);
         }
     }
 
@@ -83,9 +83,9 @@
     }
 
     public IAType getFieldType(String fieldName) {
-    	return fieldTypes[typeMap.get(fieldName)];
+        return fieldTypes[typeMap.get(fieldName)];
     }
-    
+
     @Override
     public String getDisplayName() {
         return "ARecord";