merged from asterix_stabilization r408:544

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization_temporal_fixes@545 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index 8b537a7..a573df3 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -49,8 +50,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-
-
 public class LogicalOperatorDeepCopyVisitor implements ILogicalOperatorVisitor<ILogicalOperator, ILogicalOperator> {
     private final Counter counter;
     private final LogicalExpressionDeepCopyVisitor exprDeepCopyVisitor;
@@ -378,4 +377,10 @@
     public ILogicalOperator visitSinkOperator(SinkOperator op, ILogicalOperator arg) {
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public ILogicalOperator visitExtensionOperator(ExtensionOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 1b58743..dad8d0a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -1,38 +1,40 @@
 package edu.uci.ics.asterix.algebra.operators.physical;
 
+import java.util.ArrayList;
+import java.util.List;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.functions.FunctionArgumentsConstants;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.BTreeJobGenParams;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 
-public class BTreeSearchPOperator extends TreeSearchPOperator {
+/**
+ * Contributes the runtime operator for an unnest-map representing a BTree search.
+ */
+public class BTreeSearchPOperator extends IndexSearchPOperator {
 
-    public BTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx) {
-        super(idx);
+    public BTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+        super(idx, requiresBroadcast);
     }
 
     @Override
@@ -46,68 +48,35 @@
             throws AlgebricksException {
         UnnestMapOperator unnestMap = (UnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMap.getExpressionRef().getValue();
-
-        if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
-            FunctionIdentifier fid = f.getFunctionIdentifier();
-            if (fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
-                try {
-                    contributeBtreeSearch(builder, context, unnestMap, opSchema, inputSchemas);
-                } catch (AlgebricksException e) {
-                    throw new AlgebricksException(e);
-                }
-                return;
-            }
+        if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            throw new IllegalStateException();
         }
-        throw new IllegalStateException();
-    }
-
-    private void contributeBtreeSearch(IHyracksJobBuilder builder, JobGenContext context, UnnestMapOperator unnestMap,
-            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas) throws AlgebricksException, AlgebricksException {
-        Mutable<ILogicalExpression> unnestExpr = unnestMap.getExpressionRef();
-        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr.getValue();
-
-        String idxType = getStringArgument(f, 1);
-        if (idxType != FunctionArgumentsConstants.BTREE_INDEX) {
-            throw new NotImplementedException(idxType + " indexes are not implemented.");
+        AbstractFunctionCallExpression unnestFuncExpr = (AbstractFunctionCallExpression) unnestExpr;
+        FunctionIdentifier funcIdent = unnestFuncExpr.getFunctionIdentifier();
+        if (!funcIdent.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+            return;
         }
-        String idxName = getStringArgument(f, 0);
-        String datasetName = getStringArgument(f, 2);
-
-        Pair<int[], Integer> keysLeft = getKeys(f, 3, inputSchemas);
-        Pair<int[], Integer> keysRight = getKeys(f, 4 + keysLeft.second, inputSchemas);
-
-        int p = 5 + keysLeft.second + keysRight.second;
-        boolean loInclusive = isTrue((ConstantExpression) f.getArguments().get(p).getValue());
-        boolean hiInclusive = isTrue((ConstantExpression) f.getArguments().get(p + 1).getValue());
-        buildBtreeSearch(builder, context, unnestMap, opSchema, datasetName, idxName, keysLeft.first, keysRight.first,
-                loInclusive, hiInclusive);
-    }
-
-    private boolean isTrue(ConstantExpression k) {
-        return k.getValue().isTrue();
-    }
-
-    private static void buildBtreeSearch(IHyracksJobBuilder builder, JobGenContext context, AbstractScanOperator scan,
-            IOperatorSchema opSchema, String datasetName, String indexName, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive) throws AlgebricksException, AlgebricksException {
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Trying to run btree search over external dataset (" + datasetName + ").");
+        BTreeJobGenParams jobGenParams = new BTreeJobGenParams();
+        jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
+        int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
+        int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
+        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+        AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
+        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+        IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
+        List<LogicalVariable> outputVars = unnestMap.getVariables();
+        if (jobGenParams.getRetainInput()) {
+            outputVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = AqlMetadataProvider.buildBtreeRuntime(
-                metadata, context, builder.getJobSpec(), datasetName, adecl, indexName, lowKeyFields, highKeyFields,
-                lowKeyInclusive, highKeyInclusive);
-        builder.contributeHyracksOperator(scan, btreeSearch.first);
+                builder.getJobSpec(), outputVars, opSchema, typeEnv, metadata, context, jobGenParams.getRetainInput(),
+                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+                jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive());
+        builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
 
-        ILogicalOperator srcExchange = scan.getInputs().get(0).getValue();
-        builder.contributeGraphEdge(srcExchange, 0, scan, 0);
+        ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
-
 }
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java
new file mode 100644
index 0000000..65225ca
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -0,0 +1,68 @@
+package edu.uci.ics.asterix.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+
+/**
+ * Class that embodies the commonalities between access method physical operators.
+ */
+public abstract class IndexSearchPOperator extends AbstractScanPOperator {
+
+    private final IDataSourceIndex<String, AqlSourceId> idx;
+    private final boolean requiresBroadcast;
+
+    public IndexSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+        this.idx = idx;
+        this.requiresBroadcast = requiresBroadcast;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        IDataSource<?> ds = idx.getDataSource();
+        IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
+        AbstractScanOperator as = (AbstractScanOperator) op;
+        deliveredProperties = dspp.computePropertiesVector(as.getVariables());
+    }
+
+    protected int[] getKeyIndexes(List<LogicalVariable> keyVarList, IOperatorSchema[] inputSchemas) {
+        if (keyVarList == null) {
+            return null;
+        }
+        int[] keyIndexes = new int[keyVarList.size()];
+        for (int i = 0; i < keyVarList.size(); i++) {
+            keyIndexes[i] = inputSchemas[0].findVariable(keyVarList.get(i));
+        }
+        return keyIndexes;
+    }
+    
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        if (requiresBroadcast) {
+            StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+            pv[0] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(null), null);
+            return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+        } else {
+            return super.getRequiredPropertiesForChildren(op, reqdByParent);
+        }
+    }
+}
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
new file mode 100644
index 0000000..ae2559b
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -0,0 +1,176 @@
+package edu.uci.ics.asterix.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
+import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
+import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+/**
+ * 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) {
+        super(idx, requiresBroadcast);
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        UnnestMapOperator unnestMapOp = (UnnestMapOperator) op;
+        ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+        if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            throw new IllegalStateException();
+        }
+        AbstractFunctionCallExpression unnestFuncExpr = (AbstractFunctionCallExpression) unnestExpr;
+        if (unnestFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.INDEX_SEARCH) {
+            return;
+        }
+        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());
+        // Contribute operator in hyracks job.
+        builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
+        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();
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex == null) {
+            throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
+                    + datasetName);
+        }
+        List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
+        int numSecondaryKeys = secondaryKeyFields.size();
+        if (numSecondaryKeys != 1) {
+            throw new AlgebricksException(
+                    "Cannot use "
+                            + numSecondaryKeys
+                            + " fields as a key for an inverted index. There can be only one field as a key for the inverted index index.");
+        }
+        if (itemType.getTypeTag() != ATypeTag.RECORD) {
+            throw new AlgebricksException("Only record types can be indexed.");
+        }
+        ARecordType recordType = (ARecordType) itemType;
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recordType);
+        IAType secondaryKeyType = keyPairType.first;
+        if (secondaryKeyType == null) {
+            throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+        }
+
+        // TODO: For now we assume the type of the generated tokens is the same as the indexed field.
+        // We need a better way of expressing this because tokens may be hashed, or an inverted-index may index a list type, etc.
+        ITypeTraits[] tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
+        IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
+        for (int i = 0; i < numSecondaryKeys; i++) {
+            tokenComparatorFactories[i] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
+            tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+        }
+
+        IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
+        List<LogicalVariable> outputVars = unnestMap.getVariables();
+        if (retainInput) {
+            outputVars = new ArrayList<LogicalVariable>();
+            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();
+        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.
+        IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod.getSearchModifierFactory(
+                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);
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 64d757f..27a477c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -1,18 +1,13 @@
 package edu.uci.ics.asterix.algebra.operators.physical;
 
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.functions.FunctionArgumentsConstants;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.RTreeJobGenParams;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -22,16 +17,18 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 
-public class RTreeSearchPOperator extends TreeSearchPOperator {
+/**
+ * Contributes the runtime operator for an unnest-map representing a RTree search.
+ */
+public class RTreeSearchPOperator extends IndexSearchPOperator {
 
-    public RTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx) {
-        super(idx);
+    public RTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+        super(idx, requiresBroadcast);
     }
 
     @Override
@@ -45,57 +42,31 @@
             throws AlgebricksException {
         UnnestMapOperator unnestMap = (UnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMap.getExpressionRef().getValue();
-
-        if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
-            FunctionIdentifier fid = f.getFunctionIdentifier();
-            if (fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
-                try {
-                    contributeRtreeSearch(builder, context, unnestMap, opSchema, inputSchemas);
-                } catch (AlgebricksException e) {
-                    throw new AlgebricksException(e);
-                }
-                return;
-            }
+        if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            throw new IllegalStateException();
         }
-        throw new IllegalStateException();
-    }
-
-    private void contributeRtreeSearch(IHyracksJobBuilder builder, JobGenContext context, UnnestMapOperator unnestMap,
-            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas) throws AlgebricksException, AlgebricksException {
-        Mutable<ILogicalExpression> unnestExpr = unnestMap.getExpressionRef();
-        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr.getValue();
-
-        String idxType = getStringArgument(f, 1);
-        if (idxType != FunctionArgumentsConstants.RTREE_INDEX) {
-            throw new NotImplementedException(idxType + " indexes are not implemented.");
+        AbstractFunctionCallExpression unnestFuncExpr = (AbstractFunctionCallExpression) unnestExpr;
+        FunctionIdentifier funcIdent = unnestFuncExpr.getFunctionIdentifier();
+        if (!funcIdent.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+            return;
         }
-        String idxName = getStringArgument(f, 0);
-        String datasetName = getStringArgument(f, 2);
 
-        Pair<int[], Integer> keys = getKeys(f, 3, inputSchemas);
-        buildRtreeSearch(builder, context, unnestMap, opSchema, datasetName, idxName, keys.first);
-    }
+        RTreeJobGenParams jobGenParams = new RTreeJobGenParams();
+        jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
 
-    private static void buildRtreeSearch(IHyracksJobBuilder builder, JobGenContext context, AbstractScanOperator scan,
-            IOperatorSchema opSchema, String datasetName, String indexName, int[] keyFields)
-            throws AlgebricksException, AlgebricksException {
+        int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Trying to run rtree search over external dataset (" + datasetName + ").");
+        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
         }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = AqlMetadataProvider.buildRtreeRuntime(
-                metadata, context, builder.getJobSpec(), datasetName, adecl, indexName, keyFields);
-        builder.contributeHyracksOperator(scan, rtreeSearch.first);
+                metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), dataset,
+                jobGenParams.getIndexName(), keyIndexes);
+        builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
-
-        ILogicalOperator srcExchange = scan.getInputs().get(0).getValue();
-        builder.contributeGraphEdge(srcExchange, 0, scan, 0);
+        ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
-
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/TreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/TreeSearchPOperator.java
deleted file mode 100644
index b08349d..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/TreeSearchPOperator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-package edu.uci.ics.asterix.algebra.operators.physical;
-
-
-import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.om.base.AInt32;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.base.IAObject;
-import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
-import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
-import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
-
-public abstract class TreeSearchPOperator extends AbstractScanPOperator {
-
-    private IDataSourceIndex<String, AqlSourceId> idx;
-
-    public TreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx) {
-        this.idx = idx;
-    }
-
-    @Override
-    public boolean isMicroOperator() {
-        return false;
-    }
-
-    @Override
-    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        IDataSource<?> ds = idx.getDataSource();
-        IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
-        AbstractScanOperator as = (AbstractScanOperator) op;
-        deliveredProperties = dspp.computePropertiesVector(as.getVariables());
-    }
-
-    protected Pair<int[], Integer> getKeys(AbstractFunctionCallExpression f, int k, IOperatorSchema[] inputSchemas) {
-        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) f.getArguments().get(k).getValue())
-                .getValue()).getObject();
-        int numKeys = ((AInt32) obj).getIntegerValue();
-        int[] keys = null;
-        if (numKeys > 0) {
-            keys = new int[numKeys];
-            for (int i = 0; i < numKeys; i++) {
-                LogicalVariable var = ((VariableReferenceExpression) f.getArguments().get(k + 1 + i).getValue())
-                        .getVariableReference();
-                keys[i] = inputSchemas[0].findVariable(var);
-            }
-        }
-        return new Pair<int[], Integer>(keys, numKeys);
-    }
-
-    protected String getStringArgument(AbstractFunctionCallExpression f, int k) throws AlgebricksException {
-        ILogicalExpression arg = f.getArguments().get(k).getValue();
-        if (arg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            throw new NotImplementedException("Index search calls with non-constant " + k
-                    + "-th argument are not implemented.");
-        }
-        ConstantExpression ce = (ConstantExpression) arg;
-        if (!(ce.getValue() instanceof AsterixConstantValue)) {
-            throw new AlgebricksException("Third argument to index-search() should be a string.");
-        }
-        IAObject v = ((AsterixConstantValue) ce.getValue()).getObject();
-        if (v.getType().getTypeTag() != ATypeTag.STRING) {
-            throw new AlgebricksException("Third argument to index-search() should be a string.");
-        }
-        return ((AString) v).getStringValue();
-    }
-
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
index 5e96a86..0c274b9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
@@ -1,20 +1,34 @@
 package edu.uci.ics.asterix.optimizer.base;
 
+import java.util.ArrayList;
 
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class FuzzyUtils {
 
     private final static String DEFAULT_SIM_FUNCTION = "jaccard";
-    private final static float DEFAULT_SIM_THRESHOLD = .8f;
+    private final static float JACCARD_DEFAULT_SIM_THRESHOLD = .8f;
+    private final static int EDIT_DISTANCE_DEFAULT_SIM_THRESHOLD = 1;
 
     private final static String SIM_FUNCTION_PROP_NAME = "simfunction";
     private final static String SIM_THRESHOLD_PROP_NAME = "simthreshold";
 
+    public final static String JACCARD_FUNCTION_NAME = "jaccard";
+    public final static String EDIT_DISTANCE_FUNCTION_NAME = "edit-distance";
+
     public static FunctionIdentifier getTokenizer(ATypeTag inputTag) {
         switch (inputTag) {
             case STRING:
@@ -27,8 +41,50 @@
         }
     }
 
+    public static IAObject getSimThreshold(AqlCompiledMetadataDeclarations metadata, String simFuncName) {
+        String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
+        IAObject ret = null;
+        if (simFuncName.equals(JACCARD_FUNCTION_NAME)) {
+            if (simThresholValue != null) {
+                float jaccThresh = Float.parseFloat(simThresholValue);
+                ret = new AFloat(jaccThresh);
+            } else {
+                ret = new AFloat(JACCARD_DEFAULT_SIM_THRESHOLD);
+            }
+        } else if (simFuncName.equals(EDIT_DISTANCE_FUNCTION_NAME)) {
+            if (simThresholValue != null) {
+                int edThresh = Integer.parseInt(simThresholValue);
+                ret = new AInt32(edThresh);
+            } else {
+                ret = new AFloat(EDIT_DISTANCE_DEFAULT_SIM_THRESHOLD);
+            }
+        }
+        return ret;
+    }
+
+    public static FunctionIdentifier getFunctionIdentifier(String simFuncName) {
+        if (simFuncName.equals(JACCARD_FUNCTION_NAME)) {
+            return AsterixBuiltinFunctions.SIMILARITY_JACCARD;
+        } else if (simFuncName.equals(EDIT_DISTANCE_FUNCTION_NAME)) {
+            return AsterixBuiltinFunctions.EDIT_DISTANCE;
+        }
+        return null;
+    }
+
+    public static ScalarFunctionCallExpression getComparisonExpr(String simFuncName,
+            ArrayList<Mutable<ILogicalExpression>> cmpArgs) {
+        if (simFuncName.equals(JACCARD_FUNCTION_NAME)) {
+            return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.GE),
+                    cmpArgs);
+        } else if (simFuncName.equals(EDIT_DISTANCE_FUNCTION_NAME)) {
+            return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.LE),
+                    cmpArgs);
+        }
+        return null;
+    }
+
     public static float getSimThreshold(AqlCompiledMetadataDeclarations metadata) {
-        float simThreshold = DEFAULT_SIM_THRESHOLD;
+        float simThreshold = JACCARD_DEFAULT_SIM_THRESHOLD;
         String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
         if (simThresholValue != null) {
             simThreshold = Float.parseFloat(simThresholValue);
@@ -36,6 +92,7 @@
         return simThreshold;
     }
 
+    // TODO: The default function depend on the input types. 
     public static String getSimFunction(AqlCompiledMetadataDeclarations metadata) {
         String simFunction = metadata.getPropertyValue(SIM_FUNCTION_PROP_NAME);
         if (simFunction == null) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index e45eeea..76039ab 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -29,9 +29,7 @@
 import edu.uci.ics.asterix.optimizer.rules.FuzzyJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.IfElseToSwitchCaseFunctionRule;
 import edu.uci.ics.asterix.optimizer.rules.InlineAssignIntoAggregateRule;
-import edu.uci.ics.asterix.optimizer.rules.IntroduceBTreeIndexSearchRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
-import edu.uci.ics.asterix.optimizer.rules.IntroduceRTreeIndexSearchRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
@@ -44,11 +42,15 @@
 import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
 import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
 import edu.uci.ics.asterix.optimizer.rules.SetClosedRecordConstructorsRule;
+import edu.uci.ics.asterix.optimizer.rules.SimilarityCheckRule;
 import edu.uci.ics.asterix.optimizer.rules.UnnestToDataScanRule;
+import edu.uci.ics.asterix.optimizer.rules.am.IntroduceJoinAccessMethodRule;
+import edu.uci.ics.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
@@ -106,6 +108,8 @@
         normalization.add(new ConstantFoldingRule());
         normalization.add(new UnnestToDataScanRule());
         normalization.add(new IfElseToSwitchCaseFunctionRule());
+        normalization.add(new FuzzyEqRule());
+        normalization.add(new SimilarityCheckRule());
         return normalization;
     }
 
@@ -116,6 +120,7 @@
         condPushDownAndJoinInference.add(new PushDieUpRule());
         condPushDownAndJoinInference.add(new RemoveRedundantListifyRule());
         condPushDownAndJoinInference.add(new SimpleUnnestToProductRule());
+        condPushDownAndJoinInference.add(new ComplexUnnestToProductRule());
         condPushDownAndJoinInference.add(new ComplexJoinInferenceRule());
         condPushDownAndJoinInference.add(new PushSelectIntoJoinRule());
         condPushDownAndJoinInference.add(new IntroJoinInsideSubplanRule());
@@ -155,7 +160,6 @@
         List<IAlgebraicRewriteRule> fuzzy = new LinkedList<IAlgebraicRewriteRule>();
         fuzzy.add(new FuzzyJoinRule());
         fuzzy.add(new InferTypesRule());
-        fuzzy.add(new FuzzyEqRule());
         return fuzzy;
     }
 
@@ -166,8 +170,8 @@
         consolidation.add(new InlineAssignIntoAggregateRule());
         consolidation.add(new IntroduceCombinerRule());
         consolidation.add(new CountVarToCountOneRule());
-        consolidation.add(new IntroduceBTreeIndexSearchRule());
-        consolidation.add(new IntroduceRTreeIndexSearchRule());
+        consolidation.add(new IntroduceSelectAccessMethodRule());
+        consolidation.add(new IntroduceJoinAccessMethodRule());
         consolidation.add(new RemoveUnusedAssignAndAggregateRule());
         consolidation.add(new IntroduceSecondaryIndexInsertDeleteRule());
         return consolidation;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
index 2310ba5..c1d0fea 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -104,7 +104,7 @@
     private static final JobGenContext _jobGenCtx = new JobGenContext(null, null, null,
             AqlSerializerDeserializerProvider.INSTANCE, AqlBinaryHashFunctionFactoryProvider.INSTANCE,
             AqlBinaryComparatorFactoryProvider.INSTANCE, AqlTypeTraitProvider.INSTANCE,
-            AqlBinaryBooleanInspectorImpl.INSTANCE, AqlBinaryIntegerInspector.INSTANCE,
+            AqlBinaryBooleanInspectorImpl.FACTORY, AqlBinaryIntegerInspector.FACTORY,
             AqlPrinterFactoryProvider.INSTANCE, AqlNullWriterFactory.INSTANCE, null,
             new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(AqlLogicalExpressionJobGen.INSTANCE),
             AqlExpressionTypeComputer.INSTANCE, AqlNullableTypeComputer.INSTANCE, null, null, null,
@@ -180,7 +180,7 @@
             }
             IScalarEvaluatorFactory fact = _jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
                     _emptyTypeEnv, _emptySchemas, _jobGenCtx);
-            IScalarEvaluator eval = fact.createScalarEvaluator();
+            IScalarEvaluator eval = fact.createScalarEvaluator(null);
             eval.evaluate(null, p);
             Object t = _emptyTypeEnv.getType(expr);
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
index 5adfb29..c770e9f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
@@ -7,10 +7,9 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -40,7 +39,8 @@
 public class FuzzyEqRule implements IAlgebraicRewriteRule {
 
     @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();
 
         // current operator is INNERJOIN or LEFTOUTERJOIN or SELECT
@@ -78,11 +78,16 @@
         boolean expanded = false;
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) exp;
         FunctionIdentifier fi = funcExp.getFunctionIdentifier();
-
         if (fi.equals(AsterixBuiltinFunctions.FUZZY_EQ)) {
             List<Mutable<ILogicalExpression>> inputExps = funcExp.getArguments();
 
+            // TODO: Current hack to be able to optimize selections. 
+            // We change the behavior of this rule for the specific cases of const-var, or for edit-distance functions.
+            boolean useExprAsIs = false;
+
+            String simFuncName = FuzzyUtils.getSimFunction(aqlMetadata);
             ArrayList<Mutable<ILogicalExpression>> similarityArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            List<ATypeTag> inputExprTypes = new ArrayList<ATypeTag>();
             for (int i = 0; i < 2; i++) {
                 Mutable<ILogicalExpression> inputExpRef = inputExps.get(i);
                 ILogicalExpression inputExp = inputExpRef.getValue();
@@ -93,24 +98,35 @@
                     VariableReferenceExpression inputVarRef = (VariableReferenceExpression) inputExp;
                     LogicalVariable inputVar = inputVarRef.getVariableReference();
                     IAType t = TypeHelper.getNonOptionalType((IAType) env.getVarType(inputVar));
-                    inputTypeTag = t.getTypeTag();
+                    inputExprTypes.add(t.getTypeTag());
                 } else if (inputExp.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    // TODO
-                    // AbstractFunctionCallExpression inputFuncCall =
-                    // (AbstractFunctionCallExpression) inputExp;
-                    throw new NotImplementedException();
+                    // Hack to make sure that we will add the func call as is, without wrapping a tokenizer around.
+                    inputTypeTag = ATypeTag.UNORDEREDLIST;
+                    inputExprTypes.add(inputTypeTag);
+                    useExprAsIs = true;
                 } else if (inputExp.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                    // TODO
-                    // ConstantExpression inputConst = (ConstantExpression)
-                    // inputExp;
-                    throw new NotImplementedException();
+                    ConstantExpression inputConst = (ConstantExpression) inputExp;
+                    AsterixConstantValue constVal = (AsterixConstantValue) inputConst.getValue();
+                    inputTypeTag = constVal.getObject().getType().getTypeTag();
+                    inputExprTypes.add(inputTypeTag);
+                    useExprAsIs = true;
                 } else {
                     throw new NotImplementedException();
                 }
 
+                if (simFuncName.equals(FuzzyUtils.EDIT_DISTANCE_FUNCTION_NAME)) {
+                    useExprAsIs = true;
+                }
+            }
+            // TODO: This second loop is only necessary to implement the hack.
+            for (int i = 0; i < inputExprTypes.size(); ++i) {
+                Mutable<ILogicalExpression> inputExpRef = inputExps.get(i);
+                // TODO: Change Jaccard only to accept sets. We should never have to wrap a tokenizer around.
                 // get the tokenizer (if any)
-                FunctionIdentifier tokenizer = FuzzyUtils.getTokenizer(inputTypeTag);
-                if (tokenizer == null) {
+                FunctionIdentifier tokenizer = FuzzyUtils.getTokenizer(inputExprTypes.get(i));
+                if (useExprAsIs) {
+                    similarityArgs.add(inputExpRef);
+                } else if (tokenizer == null) {
                     similarityArgs.add(inputExpRef);
                 } else {
                     ArrayList<Mutable<ILogicalExpression>> tokenizerArguments = new ArrayList<Mutable<ILogicalExpression>>();
@@ -121,49 +137,17 @@
                 }
             }
 
-            // TODO use similarity-*-check
-
-            // similarityArgs.add(new Mutable<ILogicalExpression>(new
-            // ConstantExpression(new FloatLiteral(FuzzyUtils
-            // .getSimThreshold(aqlMetadata)))));
-
-            String simFunctionName = FuzzyUtils.getSimFunction(aqlMetadata);
-
-            // FunctionIdentifier simFunctionIdentifier = new
-            // FunctionIdentifier(AsterixBuiltinFunctions.ASTERIX_NS,
-            // "similarity-" + simFunctionName + "-check");
-
-            FunctionIdentifier simFunctionIdentifier = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-                    "similarity-" + simFunctionName, true);
-
+            FunctionIdentifier simFunctionIdentifier = FuzzyUtils.getFunctionIdentifier(simFuncName);
             ScalarFunctionCallExpression similarityExp = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(simFunctionIdentifier), similarityArgs);
-
-            // ArrayList<Mutable<ILogicalExpression>> atArgs = new
-            // ArrayList<Mutable<ILogicalExpression>>();
-            // atArgs.add(new Mutable<ILogicalExpression>(similarityExp));
-            // atArgs.add(new Mutable<ILogicalExpression>(new
-            // ConstantExpression(new IntegerLiteral(0))));
-
-            // ScalarFunctionCallExpression atExp = new
-            // ScalarFunctionCallExpression(
-            // FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM),
-            // atArgs);
-
-            // expRef.setValue(atExp);
-
-            ArrayList<Mutable<ILogicalExpression>> geArgs = new ArrayList<Mutable<ILogicalExpression>>();
-            geArgs.add(new MutableObject<ILogicalExpression>(similarityExp));
-            float f = FuzzyUtils.getSimThreshold(aqlMetadata);
-            geArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AFloat(f)))));
-
-            ScalarFunctionCallExpression geExp = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.GE), geArgs);
-
-            expRef.setValue(geExp);
-
+            ArrayList<Mutable<ILogicalExpression>> cmpArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            cmpArgs.add(new MutableObject<ILogicalExpression>(similarityExp));
+            IAObject simThreshold = FuzzyUtils.getSimThreshold(aqlMetadata, simFuncName);
+            cmpArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                    simThreshold))));
+            ScalarFunctionCallExpression cmpExpr = FuzzyUtils.getComparisonExpr(simFuncName, cmpArgs);
+            expRef.setValue(cmpExpr);
             return true;
-
         } else if (fi.equals(AlgebricksBuiltinFunctions.AND) || fi.equals(AlgebricksBuiltinFunctions.OR)) {
             for (int i = 0; i < 2; i++) {
                 if (expandFuzzyEq(funcExp.getArguments().get(i), context, env, aqlMetadata)) {
@@ -171,12 +155,11 @@
                 }
             }
         }
-
         return expanded;
     }
 
     @Override
-    public boolean rewritePre( Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         return false;
     }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceBTreeIndexSearchRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceBTreeIndexSearchRule.java
deleted file mode 100644
index 9e67b2d..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceBTreeIndexSearchRule.java
+++ /dev/null
@@ -1,645 +0,0 @@
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.functions.FunctionArgumentsConstants;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.base.AInt32;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.optimizer.base.AnalysisUtil;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class IntroduceBTreeIndexSearchRule extends IntroduceTreeIndexSearchRule {
-
-    private enum LimitType {
-        LOW_INCLUSIVE, LOW_EXCLUSIVE, HIGH_INCLUSIVE, HIGH_EXCLUSIVE, EQUAL
-    }
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
-
-    /**
-     * 
-     * Matches one equality of the type var EQ const, where var is bound to an
-     * indexed field.
-     * 
-     * @throws AlgebricksException
-     * 
-     */
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
-        if (op0.getOperatorTag() == LogicalOperatorTag.SELECT) {
-            return false;
-        }
-        List<Mutable<ILogicalOperator>> children = op0.getInputs();
-        if (children == null || children.size() < 1) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef1 = children.get(0);
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef1.getValue();
-        if (context.checkIfInDontApplySet(this, op1)) {
-            return false;
-        }
-
-        if (op1.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
-        }
-        SelectOperator select = (SelectOperator) op1;
-        ILogicalExpression expr = select.getCondition().getValue();
-
-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-            FunctionIdentifier fi = fce.getFunctionIdentifier();
-            if (!AlgebricksBuiltinFunctions.isComparisonFunction(fi) && fi != AlgebricksBuiltinFunctions.AND) {
-                return false;
-            }
-        } else {
-            return false;
-        }
-
-        ArrayList<IAlgebricksConstantValue> outFilters = new ArrayList<IAlgebricksConstantValue>();
-        ArrayList<LogicalVariable> outComparedVars = new ArrayList<LogicalVariable>();
-        ArrayList<LimitType> outLimits = new ArrayList<LimitType>();
-        ArrayList<Mutable<ILogicalExpression>> outRest = new ArrayList<Mutable<ILogicalExpression>>();
-        ArrayList<Integer> foundedExprList = new ArrayList<Integer>();
-        if (!analyzeCondition(expr, outFilters, outComparedVars, outLimits, outRest, foundedExprList)) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef2 = op1.getInputs().get(0);
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
-
-        DataSourceScanOperator scanDataset;
-        Mutable<ILogicalOperator> opRef3;
-        AssignOperator assignFieldAccess = null;
-
-        if (op2.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            assignFieldAccess = (AssignOperator) op2;
-            opRef3 = op2.getInputs().get(0);
-            AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
-            if (op3.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
-                return false;
-            }
-            scanDataset = (DataSourceScanOperator) op3;
-        } else if (op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
-            scanDataset = (DataSourceScanOperator) op2;
-            opRef3 = opRef2;
-        } else {
-            return false;
-        }
-
-        String datasetName = AnalysisUtil.getDatasetName(scanDataset);
-        if (datasetName == null) {
-            return false;
-        }
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
-            throw new AlgebricksException("No metadata for dataset " + datasetName);
-        }
-        if (adecl.getDatasetType() != DatasetType.INTERNAL && adecl.getDatasetType() != DatasetType.FEED) {
-            return false;
-        }
-        IAType t = metadata.findType(adecl.getItemTypeName());
-        if (t.getTypeTag() != ATypeTag.RECORD) {
-            return false;
-        }
-        ARecordType recordType = (ARecordType) t;
-        int fldPos = 0;
-        boolean foundVar = false;
-
-        AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(adecl);
-        List<String> primIdxFields = primIdxDecl.getFieldExprs();
-
-        HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>> foundIdxExprs = new HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>>();
-
-        List<LogicalVariable> varList = (assignFieldAccess != null) ? assignFieldAccess.getVariables() : scanDataset
-                .getVariables();
-
-        for (LogicalVariable var : varList) {
-
-            String fieldName = null;
-            if (assignFieldAccess != null) {
-                AbstractLogicalExpression exprP = (AbstractLogicalExpression) assignFieldAccess.getExpressions()
-                        .get(fldPos).getValue();
-                if (exprP.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                    continue;
-                }
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) exprP;
-                FunctionIdentifier fi = fce.getFunctionIdentifier();
-
-                int fieldIndex = -1;
-                if (fi.equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
-                    ILogicalExpression nameArg = fce.getArguments().get(1).getValue();
-                    if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                        return false;
-                    }
-                    ConstantExpression cNameExpr = (ConstantExpression) nameArg;
-                    fieldName = ((AString) ((AsterixConstantValue) cNameExpr.getValue()).getObject()).getStringValue();
-                } else if (fi.equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX)) {
-                    ILogicalExpression idxArg = fce.getArguments().get(1).getValue();
-                    if (idxArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                        return false;
-                    }
-                    ConstantExpression cNameExpr = (ConstantExpression) idxArg;
-                    fieldIndex = ((AInt32) ((AsterixConstantValue) cNameExpr.getValue()).getObject()).getIntegerValue();
-                } else {
-                    return false;
-                }
-                if (fieldName == null) {
-                    fieldName = recordType.getFieldNames()[fieldIndex];
-                }
-            } else { // it is a scan, not an assign
-                if (fldPos >= varList.size() - 1) {
-                    // the last var. is the record itself, so skip it
-                    break;
-                }
-                // so the variable value is one of the partitioning fields
-                fieldName = DatasetUtils.getPartitioningExpressions(adecl).get(fldPos);
-            }
-            foundVar = findIdxExprs(adecl, primIdxFields, primIdxDecl, foundIdxExprs, outComparedVars, var, fieldName);
-            if (foundVar) {
-                break;
-            }
-            fldPos++;
-        }
-        if (!foundVar) {
-            return false;
-        }
-        AqlCompiledIndexDecl picked = findUsableIndex(adecl, foundIdxExprs);
-        boolean res;
-        if (picked == null) {
-            res = false;
-        } else {
-            res = pickIndex(opRef1, opRef3, scanDataset, assignFieldAccess, outFilters, outLimits, adecl, picked,
-                    picked == primIdxDecl, foundIdxExprs, context, outRest, foundedExprList);
-        }
-        context.addToDontApplySet(this, op1);
-        if (res) {
-            OperatorPropertiesUtil.typeOpRec(opRef1, context);
-        }
-        return res;
-    }
-
-    private boolean analyzeCondition(ILogicalExpression cond, List<IAlgebricksConstantValue> outFilters,
-            List<LogicalVariable> outComparedVars, List<LimitType> outLimits, List<Mutable<ILogicalExpression>> outRest,
-            List<Integer> foundedExprList) {
-        if (cond.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) cond;
-            FunctionIdentifier fi = fce.getFunctionIdentifier();
-            if (AlgebricksBuiltinFunctions.isComparisonFunction(fi)) {
-                return analyzeComparisonExpr(fce, outFilters, outComparedVars, outLimits);
-            }
-            boolean found = false;
-            int i = 0;
-            for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
-                outRest.add(arg);
-                ILogicalExpression e = arg.getValue();
-                if (e.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) e;
-                    if (AlgebricksBuiltinFunctions.isComparisonFunction(f2.getFunctionIdentifier())) {
-                        if (analyzeComparisonExpr(f2, outFilters, outComparedVars, outLimits)) {
-                            foundedExprList.add(i);
-                            found = true;
-                        }
-                    }
-                }
-                i++;
-            }
-            return found;
-        } else {
-            throw new IllegalStateException();
-        }
-    }
-
-    private boolean analyzeComparisonExpr(AbstractFunctionCallExpression ce, List<IAlgebricksConstantValue> outFilters,
-            List<LogicalVariable> outComparedVars, List<LimitType> outLimits) {
-        IAlgebricksConstantValue constFilterVal = null;
-        LogicalVariable fldVar = null;
-        boolean filterIsLeft = false;
-        {
-            ILogicalExpression arg1 = ce.getArguments().get(0).getValue();
-            if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                ConstantExpression ce1 = (ConstantExpression) arg1;
-                constFilterVal = ce1.getValue();
-                filterIsLeft = true;
-            } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                VariableReferenceExpression ve1 = (VariableReferenceExpression) arg1;
-                fldVar = ve1.getVariableReference();
-            } else {
-                return false;
-            }
-        }
-
-        {
-            ILogicalExpression arg2 = ce.getArguments().get(1).getValue();
-            if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                if (constFilterVal != null) {
-                    return false;
-                }
-                ConstantExpression ce2 = (ConstantExpression) arg2;
-                constFilterVal = ce2.getValue();
-            } else if (arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                if (fldVar != null) {
-                    return false;
-                }
-                VariableReferenceExpression ve2 = (VariableReferenceExpression) arg2;
-                fldVar = ve2.getVariableReference();
-            } else {
-                return false;
-            }
-        }
-
-        if (constFilterVal == null || fldVar == null) {
-            return false;
-        }
-        outFilters.add(constFilterVal);
-        outComparedVars.add(fldVar);
-        LimitType limit;
-        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(ce.getFunctionIdentifier());
-        switch (ck) {
-            case EQ: {
-                limit = LimitType.EQUAL;
-                break;
-            }
-            case GE: {
-                limit = filterIsLeft ? LimitType.HIGH_INCLUSIVE : LimitType.LOW_INCLUSIVE;
-                break;
-            }
-            case GT: {
-                limit = filterIsLeft ? LimitType.HIGH_EXCLUSIVE : LimitType.LOW_EXCLUSIVE;
-                break;
-            }
-            case LE: {
-                limit = filterIsLeft ? LimitType.LOW_INCLUSIVE : LimitType.HIGH_INCLUSIVE;
-                break;
-            }
-            case LT: {
-                limit = filterIsLeft ? LimitType.LOW_EXCLUSIVE : LimitType.HIGH_EXCLUSIVE;
-                break;
-            }
-            case NEQ: {
-                return false;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-        outLimits.add(limit);
-        return true;
-    }
-
-    private boolean pickIndex(Mutable<ILogicalOperator> opRef1, Mutable<ILogicalOperator> opRef3,
-            DataSourceScanOperator scanDataset, AssignOperator assignFieldAccess,
-            ArrayList<IAlgebricksConstantValue> filters, ArrayList<LimitType> limits, AqlCompiledDatasetDecl ddecl,
-            AqlCompiledIndexDecl picked, boolean isPrimaryIdx,
-            HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>> foundIdxExprs, IOptimizationContext context,
-            List<Mutable<ILogicalExpression>> outRest, List<Integer> foundedExprList) throws AlgebricksException {
-        int numKeys = picked.getFieldExprs().size();
-        IAlgebricksConstantValue[] loFilter = new IAlgebricksConstantValue[numKeys];
-        IAlgebricksConstantValue[] hiFilter = new IAlgebricksConstantValue[numKeys];
-        LimitType[] loLimit = new LimitType[numKeys];
-        LimitType[] hiLimit = new LimitType[numKeys];
-        boolean[] loInclusive = new boolean[numKeys];
-        boolean[] hiInclusive = new boolean[numKeys];
-        List<Pair<String, Integer>> psiList = foundIdxExprs.get(picked);
-
-        boolean couldntFigureOut = false;
-        for (Pair<String, Integer> psi : psiList) {
-            int keyPos = indexOf(psi.first, picked.getFieldExprs());
-            if (keyPos < 0) {
-                throw new InternalError();
-            }
-            if (!outRest.isEmpty()) {
-                int exprIdxToBeDeleted = foundedExprList.get(psi.second);
-                outRest.set(exprIdxToBeDeleted, null);
-            }
-            LimitType lim = limits.get(psi.second);
-            boolean out = false;
-            switch (lim) {
-                case EQUAL: {
-                    if (loLimit[keyPos] == null && hiLimit[keyPos] == null) {
-                        loLimit[keyPos] = hiLimit[keyPos] = lim;
-                        loInclusive[keyPos] = hiInclusive[keyPos] = true;
-                        loFilter[keyPos] = hiFilter[keyPos] = filters.get(psi.second);
-                    } else {
-                        couldntFigureOut = true;
-                    }
-                    // hmmm, we would need an inference system here
-                    out = true;
-                    break;
-                }
-                case HIGH_EXCLUSIVE: {
-                    if (hiLimit[keyPos] == null || (hiLimit[keyPos] != null && hiInclusive[keyPos])) {
-                        hiLimit[keyPos] = lim;
-                        hiFilter[keyPos] = filters.get(psi.second);
-                        hiInclusive[keyPos] = false;
-                    } else {
-                        couldntFigureOut = true;
-                        out = true;
-                    }
-                    break;
-                }
-                case HIGH_INCLUSIVE: {
-                    if (hiLimit[keyPos] == null) {
-                        hiLimit[keyPos] = lim;
-                        hiFilter[keyPos] = filters.get(psi.second);
-                        hiInclusive[keyPos] = true;
-                    } else {
-                        couldntFigureOut = true;
-                        out = true;
-                    }
-                    break;
-                }
-                case LOW_EXCLUSIVE: {
-                    if (loLimit[keyPos] == null || (loLimit[keyPos] != null && loInclusive[keyPos])) {
-                        loLimit[keyPos] = lim;
-                        loFilter[keyPos] = filters.get(psi.second);
-                        loInclusive[keyPos] = false;
-                    } else {
-                        couldntFigureOut = true;
-                        out = true;
-                    }
-                    break;
-                }
-                case LOW_INCLUSIVE: {
-                    if (loLimit[keyPos] == null) {
-                        loLimit[keyPos] = lim;
-                        loFilter[keyPos] = filters.get(psi.second);
-                        loInclusive[keyPos] = true;
-                    } else {
-                        couldntFigureOut = true;
-                        out = true;
-                    }
-                    break;
-                }
-                default: {
-                    throw new IllegalStateException();
-                }
-            }
-            if (out) {
-                break;
-            }
-        }
-        if (couldntFigureOut) {
-            return false;
-        }
-
-        // rule out the cases unsupported by the current btree search
-        // implementation
-        for (int i = 1; i < numKeys; i++) {
-            if (loInclusive[i] != loInclusive[0] || hiInclusive[i] != hiInclusive[0]) {
-                return false;
-            }
-            if (loLimit[0] == null && loLimit[i] != null || loLimit[0] != null && loLimit[i] == null) {
-                return false;
-            }
-            if (hiLimit[0] == null && hiLimit[i] != null || hiLimit[0] != null && hiLimit[i] == null) {
-                return false;
-            }
-        }
-        if (loLimit[0] == null) {
-            loInclusive[0] = true;
-        }
-        if (hiLimit[0] == null) {
-            hiInclusive[0] = true;
-        }
-
-        ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
-        ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
-        ArrayList<Mutable<ILogicalExpression>> rangeSearchFunArgs = new ArrayList<Mutable<ILogicalExpression>>();
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(picked.getIndexName())));
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(FunctionArgumentsConstants.BTREE_INDEX)));
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-
-        if (loLimit[0] != null) {
-            Mutable<ILogicalExpression> nkRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                    new AsterixConstantValue(new AInt32(numKeys))));
-            rangeSearchFunArgs.add(nkRef);
-            for (int i = 0; i < numKeys; i++) {
-                LogicalVariable lokVar = context.newVar();
-                keyVarList.add(lokVar);
-                keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(loFilter[i])));
-                Mutable<ILogicalExpression> loRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                        lokVar));
-                rangeSearchFunArgs.add(loRef);
-            }
-        } else {
-            Mutable<ILogicalExpression> zeroRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                    new AsterixConstantValue(new AInt32(0))));
-            rangeSearchFunArgs.add(zeroRef);
-        }
-
-        if (hiLimit[0] != null) {
-            Mutable<ILogicalExpression> nkRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                    new AsterixConstantValue(new AInt32(numKeys))));
-            rangeSearchFunArgs.add(nkRef);
-            for (int i = 0; i < numKeys; i++) {
-                LogicalVariable hikVar = context.newVar();
-                keyVarList.add(hikVar);
-                keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(hiFilter[i])));
-                Mutable<ILogicalExpression> hiRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                        hikVar));
-                rangeSearchFunArgs.add(hiRef);
-            }
-        } else {
-            Mutable<ILogicalExpression> zeroRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                    new AsterixConstantValue(new AInt32(0))));
-            rangeSearchFunArgs.add(zeroRef);
-        }
-
-        ILogicalExpression loExpr = loInclusive[0] ? ConstantExpression.TRUE : ConstantExpression.FALSE;
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(loExpr));
-        ILogicalExpression hiExpr = hiInclusive[0] ? ConstantExpression.TRUE : ConstantExpression.FALSE;
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(hiExpr));
-
-        AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
-        assignSearchKeys.getInputs().add(scanDataset.getInputs().get(0));
-        assignSearchKeys.setExecutionMode(scanDataset.getExecutionMode());
-
-        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        UnnestingFunctionCallExpression rangeSearchFun = new UnnestingFunctionCallExpression(finfo, rangeSearchFunArgs);
-        rangeSearchFun.setReturnsUniqueValues(true);
-
-        List<LogicalVariable> primIdxVarList = scanDataset.getVariables();
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
-
-        UnnestMapOperator primIdxUnnestMap;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-
-        String itemTypeName = ddecl.getItemTypeName();
-        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
-        if (!isPrimaryIdx) {
-            ArrayList<LogicalVariable> secIdxPrimKeysVarList = new ArrayList<LogicalVariable>(numPrimaryKeys);
-            for (int i = 0; i < numPrimaryKeys; i++) {
-                secIdxPrimKeysVarList.add(context.newVar());
-            }
-            ArrayList<LogicalVariable> secIdxUnnestVars = new ArrayList<LogicalVariable>(numKeys
-                    + secIdxPrimKeysVarList.size());
-            for (int i = 0; i < numKeys; i++) {
-                secIdxUnnestVars.add(context.newVar());
-            }
-            secIdxUnnestVars.addAll(secIdxPrimKeysVarList);
-            UnnestMapOperator secIdxUnnest = new UnnestMapOperator(secIdxUnnestVars, new MutableObject<ILogicalExpression>(
-                    rangeSearchFun), secondaryIndexTypes(ddecl, picked, itemType));
-            secIdxUnnest.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
-            secIdxUnnest.setExecutionMode(ExecutionMode.PARTITIONED);
-
-            OrderOperator order = new OrderOperator();
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v));
-                order.getOrderExpressions().add(
-                        new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
-            }
-            order.getInputs().add(new MutableObject<ILogicalOperator>(secIdxUnnest));
-            order.setExecutionMode(ExecutionMode.LOCAL);
-
-            List<Mutable<ILogicalExpression>> argList2 = new ArrayList<Mutable<ILogicalExpression>>();
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(FunctionArgumentsConstants.BTREE_INDEX)));
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-            argList2.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
-                    numPrimaryKeys)))));
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                argList2.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            argList2.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
-                    numPrimaryKeys)))));
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                argList2.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            argList2.add(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
-            argList2.add(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
-            IFunctionInfo finfoSearch2 = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-            AbstractFunctionCallExpression searchPrimIdxFun = new ScalarFunctionCallExpression(finfoSearch2, argList2);
-            primIdxUnnestMap = new UnnestMapOperator(primIdxVarList, new MutableObject<ILogicalExpression>(searchPrimIdxFun),
-                    primaryIndexTypes(metadata, ddecl, itemType));
-            primIdxUnnestMap.getInputs().add(new MutableObject<ILogicalOperator>(order));
-        } else {
-            primIdxUnnestMap = new UnnestMapOperator(primIdxVarList, new MutableObject<ILogicalExpression>(rangeSearchFun),
-                    primaryIndexTypes(metadata, ddecl, itemType));
-            primIdxUnnestMap.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
-        }
-
-        primIdxUnnestMap.setExecutionMode(ExecutionMode.PARTITIONED);
-
-        validateRemainingPreds(outRest);
-        if (!outRest.isEmpty()) {
-            ILogicalExpression pulledCond = makeCondition(outRest, context);
-            SelectOperator selectRest = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
-            if (assignFieldAccess != null) {
-                opRef3.setValue(primIdxUnnestMap);
-                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(assignFieldAccess));
-            } else {
-                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(primIdxUnnestMap));
-            }
-            selectRest.setExecutionMode(((AbstractLogicalOperator) opRef1.getValue()).getExecutionMode());
-            opRef1.setValue(selectRest);
-        } else {
-            primIdxUnnestMap.setExecutionMode(ExecutionMode.PARTITIONED);
-            if (assignFieldAccess != null) {
-                opRef3.setValue(primIdxUnnestMap);
-                opRef1.setValue(assignFieldAccess);
-            } else {
-                opRef1.setValue(primIdxUnnestMap);
-            }
-        }
-
-        return true;
-    }
-
-    private void validateRemainingPreds(List<Mutable<ILogicalExpression>> predList) {
-        for (int i = 0; i < predList.size();) {
-            if (predList.get(i) == null) {
-                predList.remove(i);
-            } else {
-                i++;
-            }
-        }
-    }
-
-    private ILogicalExpression makeCondition(List<Mutable<ILogicalExpression>> predList, IOptimizationContext context) {
-        if (predList.size() > 1) {
-            IFunctionInfo finfo = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
-            return new ScalarFunctionCallExpression(finfo, predList);
-        } else {
-            return predList.get(0).getValue();
-        }
-    }
-
-    private static List<Object> secondaryIndexTypes(AqlCompiledDatasetDecl ddecl, AqlCompiledIndexDecl acid,
-            ARecordType itemType) throws AlgebricksException {
-        List<Object> types = new ArrayList<Object>();
-        for (String sk : acid.getFieldExprs()) {
-            Pair<IAType, Boolean> keyPair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(sk, itemType);
-            types.add(keyPair.first);
-        }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : DatasetUtils
-                .getPartitioningFunctions(ddecl)) {
-            types.add(t.third);
-        }
-        return types;
-    }
-
-    private <T> int indexOf(T value, List<T> coll) {
-        int i = 0;
-        for (T member : coll) {
-            if (member.equals(value)) {
-                return i;
-            }
-            i++;
-        }
-        return -1;
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRTreeIndexSearchRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRTreeIndexSearchRule.java
deleted file mode 100644
index de482dd..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRTreeIndexSearchRule.java
+++ /dev/null
@@ -1,395 +0,0 @@
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.functions.FunctionArgumentsConstants;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.base.AInt32;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.asterix.optimizer.base.AnalysisUtil;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class IntroduceRTreeIndexSearchRule extends IntroduceTreeIndexSearchRule {
-
-    /**
-     * 
-     * Matches spatial-intersect(var, spatialObject) , where var is bound to an
-     * indexed field, and spatialObject is point, line, polygon, circle or
-     * rectangle
-     * 
-     * @throws AlgebricksException
-     * 
-     */
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
-        if (context.checkIfInDontApplySet(this, op1)) {
-            return false;
-        }
-        if (op1.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
-        }
-
-        SelectOperator select = (SelectOperator) op1;
-        ILogicalExpression expr = select.getCondition().getValue();
-
-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-            FunctionIdentifier fi = fce.getFunctionIdentifier();
-            if (!AsterixBuiltinFunctions.isSpatialFilterFunction(fi) && !fi.equals(AlgebricksBuiltinFunctions.AND)) {
-                return false;
-            }
-        } else {
-            return false;
-        }
-        ArrayList<IAlgebricksConstantValue> outFilters = new ArrayList<IAlgebricksConstantValue>();
-        ArrayList<LogicalVariable> outComparedVars = new ArrayList<LogicalVariable>();
-
-        if (!analyzeCondition(expr, outFilters, outComparedVars)) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef2 = op1.getInputs().get(0);
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
-
-        DataSourceScanOperator scanDataset;
-        Mutable<ILogicalOperator> opRef3;
-        AssignOperator assignFieldAccess = null;
-
-        if (op2.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            assignFieldAccess = (AssignOperator) op2;
-            opRef3 = op2.getInputs().get(0);
-            AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
-            if (op3.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
-                return false;
-            }
-            scanDataset = (DataSourceScanOperator) op3;
-        } else {
-            return false;
-        }
-
-        String datasetName = AnalysisUtil.getDatasetName(scanDataset);
-        if (datasetName == null) {
-            return false;
-        }
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
-            throw new AlgebricksException("No metadata for dataset " + datasetName);
-        }
-        if (adecl.getDatasetType() != DatasetType.INTERNAL && adecl.getDatasetType() != DatasetType.FEED) {
-            return false;
-        }
-        IAType t = metadata.findType(adecl.getItemTypeName());
-        if (t.getTypeTag() != ATypeTag.RECORD) {
-            return false;
-        }
-        ARecordType recordType = (ARecordType) t;
-        int fldPos = 0;
-        boolean foundVar = false;
-        int dimension = 0;
-        AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(adecl);
-        List<String> primIdxFields = primIdxDecl.getFieldExprs();
-
-        HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>> foundIdxExprs = new HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>>();
-
-        List<LogicalVariable> varList = assignFieldAccess.getVariables();
-
-        for (LogicalVariable var : varList) {
-
-            String fieldName = null;
-            AbstractLogicalExpression exprP = (AbstractLogicalExpression) assignFieldAccess.getExpressions()
-                    .get(fldPos).getValue();
-            if (exprP.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                continue;
-            }
-            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) exprP;
-            FunctionIdentifier fi = fce.getFunctionIdentifier();
-
-            int fieldIndex = -1;
-
-            if (fi.equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
-                ILogicalExpression nameArg = fce.getArguments().get(1).getValue();
-                if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                    return false;
-                }
-                ConstantExpression cNameExpr = (ConstantExpression) nameArg;
-                fieldName = ((AString) ((AsterixConstantValue) cNameExpr.getValue()).getObject()).getStringValue();
-            } else if (fi.equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX)) {
-                ILogicalExpression idxArg = fce.getArguments().get(1).getValue();
-                if (idxArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                    return false;
-                }
-                ConstantExpression cNameExpr = (ConstantExpression) idxArg;
-                fieldIndex = ((AInt32) ((AsterixConstantValue) cNameExpr.getValue()).getObject()).getIntegerValue();
-            } else {
-                return false;
-            }
-
-            if (fieldName == null) {
-                fieldName = recordType.getFieldNames()[fieldIndex];
-            }
-
-            foundVar = findIdxExprs(adecl, primIdxFields, primIdxDecl, foundIdxExprs, outComparedVars, var, fieldName);
-            if (foundVar) {
-                Pair<IAType, Boolean> spatialTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(fieldName, recordType);
-                dimension = NonTaggedFormatUtil.getNumDimensions(spatialTypePair.first.getTypeTag());
-            }
-
-            fldPos++;
-        }
-        if (!foundVar) {
-            return false;
-        }
-        AqlCompiledIndexDecl picked = findUsableIndex(adecl, foundIdxExprs);
-        boolean res;
-        if (picked == null) {
-            res = false;
-        } else {
-            res = pickIndex(opRef3, scanDataset, assignFieldAccess, outFilters, adecl, picked, picked == primIdxDecl,
-                    context, dimension);
-        }
-        context.addToDontApplySet(this, op1);
-        if (res) {
-            OperatorPropertiesUtil.typeOpRec(opRef, context);
-        }
-        return res;
-    }
-
-    private boolean analyzeCondition(ILogicalExpression cond, List<IAlgebricksConstantValue> outFilters,
-            List<LogicalVariable> outComparedVars) {
-        if (cond.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) cond;
-            FunctionIdentifier fi = fce.getFunctionIdentifier();
-            if (AsterixBuiltinFunctions.isSpatialFilterFunction(fi)) {
-                return analyzeSpatialFilterExpr(fce, outFilters, outComparedVars);
-            }
-            boolean found = false;
-
-            for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
-                ILogicalExpression e = arg.getValue();
-                if (e.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) e;
-                    if (AsterixBuiltinFunctions.isSpatialFilterFunction(f2.getFunctionIdentifier())) {
-                        if (analyzeSpatialFilterExpr(f2, outFilters, outComparedVars)) {
-                            found = true;
-                        }
-                    }
-                }
-            }
-            return found;
-        } else {
-            throw new IllegalStateException();
-        }
-    }
-
-    private boolean analyzeSpatialFilterExpr(AbstractFunctionCallExpression ce,
-            List<IAlgebricksConstantValue> outFilters, List<LogicalVariable> outComparedVars) {
-        IAlgebricksConstantValue constFilterVal = null;
-        LogicalVariable fldVar = null;
-        ILogicalExpression arg1 = ce.getArguments().get(0).getValue();
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            ConstantExpression ce1 = (ConstantExpression) arg1;
-            constFilterVal = ce1.getValue();
-        } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-            VariableReferenceExpression ve1 = (VariableReferenceExpression) arg1;
-            fldVar = ve1.getVariableReference();
-        } else {
-            return false;
-        }
-        ILogicalExpression arg2 = ce.getArguments().get(1).getValue();
-        if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            if (constFilterVal != null) {
-                return false;
-            }
-            ConstantExpression ce2 = (ConstantExpression) arg2;
-            constFilterVal = ce2.getValue();
-        } else if (arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-            if (fldVar != null) {
-                return false;
-            }
-            VariableReferenceExpression ve2 = (VariableReferenceExpression) arg2;
-            fldVar = ve2.getVariableReference();
-        } else {
-            return false;
-        }
-
-        if (constFilterVal == null || fldVar == null) {
-            return false;
-        }
-
-        outFilters.add(constFilterVal);
-        outComparedVars.add(fldVar);
-        return true;
-    }
-
-    private boolean pickIndex(Mutable<ILogicalOperator> opRef3, DataSourceScanOperator scanDataset,
-            AssignOperator assignFieldAccess, ArrayList<IAlgebricksConstantValue> filters,
-            AqlCompiledDatasetDecl ddecl, AqlCompiledIndexDecl picked, boolean isPrimaryIdx,
-            IOptimizationContext context, int dimension) throws AlgebricksException {
-        int numKeys = dimension * 2;
-
-        ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
-        ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
-        ArrayList<Mutable<ILogicalExpression>> rangeSearchFunArgs = new ArrayList<Mutable<ILogicalExpression>>();
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(picked.getIndexName())));
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(FunctionArgumentsConstants.RTREE_INDEX)));
-        rangeSearchFunArgs.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-
-        Mutable<ILogicalExpression> nkRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                new AsterixConstantValue(new AInt32(numKeys))));
-        rangeSearchFunArgs.add(nkRef);
-        for (int i = 0; i < numKeys; i++) {
-            LogicalVariable keyVar = context.newVar();
-            keyVarList.add(keyVar);
-
-            AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CREATE_MBR));
-            createMBR.getArguments().add(new MutableObject<ILogicalExpression>(new ConstantExpression(filters.get(0))));
-            createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                            new AInt32(dimension)))));
-            createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
-            keyExprList.add(new MutableObject<ILogicalExpression>(createMBR));
-            Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                    keyVar));
-            rangeSearchFunArgs.add(keyVarRef);
-        }
-
-        AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
-        assignSearchKeys.getInputs().add(scanDataset.getInputs().get(0));
-        assignSearchKeys.setExecutionMode(scanDataset.getExecutionMode());
-
-        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        UnnestingFunctionCallExpression rangeSearchFun = new UnnestingFunctionCallExpression(finfo, rangeSearchFunArgs);
-        rangeSearchFun.setReturnsUniqueValues(true);
-
-        List<LogicalVariable> primIdxVarList = scanDataset.getVariables();
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
-
-        UnnestMapOperator primIdxUnnestMap;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-
-        String itemTypeName = ddecl.getItemTypeName();
-        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
-        if (!isPrimaryIdx) {
-            ArrayList<LogicalVariable> secIdxPrimKeysVarList = new ArrayList<LogicalVariable>(numPrimaryKeys);
-            for (int i = 0; i < numPrimaryKeys; i++) {
-                secIdxPrimKeysVarList.add(context.newVar());
-            }
-            ArrayList<LogicalVariable> secIdxUnnestVars = new ArrayList<LogicalVariable>(numKeys
-                    + secIdxPrimKeysVarList.size());
-            for (int i = 0; i < numKeys; i++) {
-                secIdxUnnestVars.add(context.newVar());
-            }
-            secIdxUnnestVars.addAll(secIdxPrimKeysVarList);
-            UnnestMapOperator secIdxUnnest = new UnnestMapOperator(secIdxUnnestVars, new MutableObject<ILogicalExpression>(
-                    rangeSearchFun), secondaryIndexTypes(ddecl, picked, itemType, numKeys));
-            secIdxUnnest.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
-            secIdxUnnest.setExecutionMode(ExecutionMode.PARTITIONED);
-
-            OrderOperator order = new OrderOperator();
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v));
-                order.getOrderExpressions().add(
-                        new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
-            }
-            order.getInputs().add(new MutableObject<ILogicalOperator>(secIdxUnnest));
-            order.setExecutionMode(ExecutionMode.LOCAL);
-
-            List<Mutable<ILogicalExpression>> argList2 = new ArrayList<Mutable<ILogicalExpression>>();
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(FunctionArgumentsConstants.BTREE_INDEX)));
-            argList2.add(new MutableObject<ILogicalExpression>(mkStrConstExpr(ddecl.getName())));
-            argList2.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
-                    numPrimaryKeys)))));
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                argList2.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            argList2.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
-                    numPrimaryKeys)))));
-            for (LogicalVariable v : secIdxPrimKeysVarList) {
-                argList2.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            argList2.add(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
-            argList2.add(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
-            IFunctionInfo finfoSearch2 = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-            AbstractFunctionCallExpression searchPrimIdxFun = new ScalarFunctionCallExpression(finfoSearch2, argList2);
-            primIdxUnnestMap = new UnnestMapOperator(primIdxVarList, new MutableObject<ILogicalExpression>(searchPrimIdxFun),
-                    primaryIndexTypes(metadata, ddecl, itemType));
-            primIdxUnnestMap.getInputs().add(new MutableObject<ILogicalOperator>(order));
-
-        } else {
-            return false;
-        }
-
-        primIdxUnnestMap.setExecutionMode(ExecutionMode.PARTITIONED);
-        opRef3.setValue(primIdxUnnestMap);
-
-        return true;
-    }
-
-    private static List<Object> secondaryIndexTypes(AqlCompiledDatasetDecl ddecl, AqlCompiledIndexDecl acid,
-            ARecordType itemType, int numKeys) throws AlgebricksException {
-        List<Object> types = new ArrayList<Object>();
-        Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(acid.getFieldExprs().get(0), itemType);
-        IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyTypePair.first.getTypeTag());
-
-        for (int i = 0; i < numKeys; i++) {
-            types.add(nestedKeyType);
-        }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : DatasetUtils
-                .getPartitioningFunctions(ddecl)) {
-            types.add(t.third);
-        }
-        return types;
-    }
-}
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 dc5d659..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
@@ -8,14 +8,13 @@
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlIndex;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -80,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();
             }
         }
@@ -92,30 +91,32 @@
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
         String datasetName = datasetSource.getId().getDatasetName();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
         }
 
         List<LogicalVariable> projectVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getUsedVariables(op1, projectVars);
         // Create operators for secondary index insert/delete.
-        String itemTypeName = adecl.getItemTypeName();
+        String itemTypeName = dataset.getItemTypeName();
         IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
-        List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
-        if (secondaryIndexes.isEmpty()) {
-            return false;
-        }
+        List<Index> indexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         ILogicalOperator currentTop = op1;
-        for (AqlCompiledIndexDecl index : secondaryIndexes) {
-            List<String> secondaryKeyFields = index.getFieldExprs();
+        boolean hasSecondaryIndex = false;
+        for (Index index : indexes) {
+            if (!index.isSecondaryIndex()) {
+                continue;
+            }
+            hasSecondaryIndex = true;
+            List<String> secondaryKeyFields = index.getKeyFieldNames();
             List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
             List<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
@@ -146,7 +147,7 @@
             project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             context.computeAndSetTypeEnvironmentForOperator(project);
             context.computeAndSetTypeEnvironmentForOperator(assign);
-            if (index.getKind() == IndexKind.BTREE) {
+            if (index.getIndexType() == IndexType.BTREE) {
                 for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
                     secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
                             secondaryKeyVar)));
@@ -160,9 +161,9 @@
                 indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                 currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
-            } else if (index.getKind() == IndexKind.RTREE) {
-                Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                        secondaryKeyFields.get(0), recType);
+            } else if (index.getIndexType() == IndexType.RTREE) {
+                Pair<IAType, Boolean> keyPairType = Index
+                        .getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
                 IAType spatialType = keyPairType.first;
                 int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
                 int numKeys = dimension * 2;
@@ -199,11 +200,13 @@
                 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);
             }
-
+        }
+        if (!hasSecondaryIndex) {
+            return false;
         }
         op0.getInputs().clear();
         op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
@@ -219,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)));
@@ -242,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/IntroduceTreeIndexSearchRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTreeIndexSearchRule.java
deleted file mode 100644
index be00f6d..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTreeIndexSearchRule.java
+++ /dev/null
@@ -1,108 +0,0 @@
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public abstract class IntroduceTreeIndexSearchRule implements IAlgebraicRewriteRule {
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
-
-    /**
-     * just picks the first index for which all the expressions are mentioned
-     */
-    protected AqlCompiledIndexDecl findUsableIndex(AqlCompiledDatasetDecl ddecl,
-            HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>> foundIdxExprs) {
-        for (AqlCompiledIndexDecl acid : foundIdxExprs.keySet()) {
-            List<Pair<String, Integer>> psiList = foundIdxExprs.get(acid);
-            boolean allUsed = true;
-            for (String keyField : acid.getFieldExprs()) {
-                boolean foundKf = false;
-                for (Pair<String, Integer> psi : psiList) {
-                    if (psi.first.equals(keyField)) {
-                        foundKf = true;
-                        break;
-                    }
-                }
-                if (!foundKf) {
-                    allUsed = false;
-                    break;
-                }
-            }
-            if (allUsed) {
-                return acid;
-            }
-        }
-        return null;
-    }
-
-    protected static ConstantExpression mkStrConstExpr(String str) {
-        return new ConstantExpression(new AsterixConstantValue(new AString(str)));
-    }
-
-    protected boolean findIdxExprs(AqlCompiledDatasetDecl ddecl, List<String> primIdxFields,
-            AqlCompiledIndexDecl primIdxDecl, HashMap<AqlCompiledIndexDecl, List<Pair<String, Integer>>> foundIdxExprs,
-            ArrayList<LogicalVariable> comparedVars, LogicalVariable var, String fieldName) {
-        boolean foundVar = false;
-        List<AqlCompiledIndexDecl> idxList = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(ddecl, fieldName);
-        if (primIdxFields.contains(fieldName)) {
-            if (idxList == null) {
-                idxList = new ArrayList<AqlCompiledIndexDecl>(1);
-            }
-            idxList.add(primIdxDecl);
-        }
-        if (idxList != null) {
-            foundVar = true;
-            for (AqlCompiledIndexDecl idx : idxList) {
-                List<Pair<String, Integer>> psi = foundIdxExprs.get(idx);
-                if (psi == null) {
-                    psi = new ArrayList<Pair<String, Integer>>();
-                    foundIdxExprs.put(idx, psi);
-                }
-                int varPos = 0;
-                for (LogicalVariable v : comparedVars) {
-                    if (v == var) {
-                        psi.add(new Pair<String, Integer>(fieldName, varPos));
-                    }
-                    varPos++;
-                }
-            }
-        }
-        return foundVar;
-    }
-
-    protected static List<Object> primaryIndexTypes(AqlCompiledMetadataDeclarations metadata,
-            AqlCompiledDatasetDecl ddecl, IAType itemType) {
-        List<Object> types = new ArrayList<Object>();
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(ddecl);
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : partitioningFunctions) {
-            types.add(t.third);
-        }
-        types.add(itemType);
-        return types;
-    }
-
-}
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 5a3d3b8..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
@@ -11,11 +11,11 @@
 import edu.uci.ics.asterix.algebra.base.AsterixOperatorAnnotations;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
@@ -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;
@@ -116,11 +117,11 @@
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
         AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(asid.getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
         }
-        if (adecl.getDatasetType() != DatasetType.INTERNAL && adecl.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             return false;
         }
         ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
@@ -134,7 +135,7 @@
             fldName = ((AString) obj).getStringValue();
         } else {
             int pos = ((AInt32) obj).getIntegerValue();
-            String tName = adecl.getItemTypeName();
+            String tName = dataset.getItemTypeName();
             IAType t = metadata.findType(tName);
             if (t.getTypeTag() != ATypeTag.RECORD) {
                 return false;
@@ -146,9 +147,16 @@
             fldName = rt.getFieldNames()[pos];
         }
 
-        List<AqlCompiledIndexDecl> idxList = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(adecl, fldName);
+        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        boolean hasSecondaryIndex = false;
+        for (Index index : datasetIndexes) {
+            if (index.isSecondaryIndex()) {
+                hasSecondaryIndex = true;
+                break;
+            }
+        }
 
-        return idxList != null && !idxList.isEmpty();
+        return hasSecondaryIndex;
     }
 
     private boolean tryingToPushThroughSelectionWithSameDataSource(AssignOperator access, AbstractLogicalOperator op2) {
@@ -285,12 +293,12 @@
                             AqlSourceId asid = dataSource.getId();
                             AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                             AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                            AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
-                            if (adecl == null) {
+                            Dataset dataset = metadata.findDataset(asid.getDatasetName());
+                            if (dataset == null) {
                                 throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
                             }
-                            if (adecl.getDatasetType() != DatasetType.INTERNAL
-                                    && adecl.getDatasetType() != DatasetType.FEED) {
+                            if (dataset.getDatasetType() != DatasetType.INTERNAL
+                                    && dataset.getDatasetType() != DatasetType.FEED) {
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
                             }
@@ -301,7 +309,7 @@
                                 fldName = ((AString) obj).getStringValue();
                             } else {
                                 int pos = ((AInt32) obj).getIntegerValue();
-                                String tName = adecl.getItemTypeName();
+                                String tName = dataset.getItemTypeName();
                                 IAType t = metadata.findType(tName);
                                 if (t.getTypeTag() != ATypeTag.RECORD) {
                                     return false;
@@ -313,7 +321,7 @@
                                 }
                                 fldName = rt.getFieldNames()[pos];
                             }
-                            int p = DatasetUtils.getPositionOfPartitioningKeyField(adecl, fldName);
+                            int p = DatasetUtils.getPositionOfPartitioningKeyField(dataset, fldName);
                             if (p < 0) { // not one of the partitioning fields
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index a1f6edc..74f790f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -7,13 +7,13 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.InvertedIndexPOperator;
 import edu.uci.ics.asterix.algebra.operators.physical.RTreeSearchPOperator;
-import edu.uci.ics.asterix.common.functions.FunctionArgumentsConstants;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -48,7 +47,8 @@
 public class SetAsterixPhysicalOperatorsRule implements IAlgebraicRewriteRule {
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -140,41 +140,47 @@
                 case UNNEST_MAP: {
                     UnnestMapOperator unnestMap = (UnnestMapOperator) op;
                     ILogicalExpression unnestExpr = unnestMap.getExpressionRef().getValue();
-                    boolean notSet = true;
                     if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                         AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                         FunctionIdentifier fid = f.getFunctionIdentifier();
-                        if (fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
-                            notSet = false;
-                            AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-                            ConstantExpression ce0 = (ConstantExpression) f.getArguments().get(0).getValue();
-                            String indexId = ((AString) ((AsterixConstantValue) ce0.getValue()).getObject())
-                                    .getStringValue();
-                            ConstantExpression ce2 = (ConstantExpression) f.getArguments().get(2).getValue();
-                            String datasetName = ((AString) ((AsterixConstantValue) ce2.getValue()).getObject())
-                                    .getStringValue();
-                            String dvName = mp.getMetadataDeclarations().getDataverseName();
-                            AqlSourceId dataSourceId = new AqlSourceId(dvName, datasetName);
-                            IDataSourceIndex<String, AqlSourceId> dsi = mp.findDataSourceIndex(indexId, dataSourceId);
-                            if (dsi == null) {
-                                throw new AlgebricksException("Could not find index " + indexId + " for dataset "
-                                        + dataSourceId);
+                        if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                            throw new IllegalStateException();
+                        }
+                        AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                        jobGenParams.readFromFuncArgs(f.getArguments());
+                        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+                        String dataverseName = mp.getMetadataDeclarations().getDataverseName();
+                        AqlSourceId dataSourceId = new AqlSourceId(dataverseName, jobGenParams.getDatasetName());
+                        IDataSourceIndex<String, AqlSourceId> dsi = mp.findDataSourceIndex(jobGenParams.getIndexName(),
+                                dataSourceId);
+                        if (dsi == null) {
+                            throw new AlgebricksException("Could not find index " + jobGenParams.getIndexName()
+                                    + " for dataset " + dataSourceId);
+                        }
+                        IndexType indexType = jobGenParams.getIndexType();
+                        boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
+                        switch (indexType) {
+                            case BTREE: {
+                                op.setPhysicalOperator(new BTreeSearchPOperator(dsi, requiresBroadcast));
+                                break;
                             }
-                            ConstantExpression ce1 = (ConstantExpression) f.getArguments().get(1).getValue();
-                            String indexType = ((AString) ((AsterixConstantValue) ce1.getValue()).getObject())
-                                    .getStringValue();
-                            if (indexType == FunctionArgumentsConstants.BTREE_INDEX) {
-                                op.setPhysicalOperator(new BTreeSearchPOperator(dsi));
-                            } else if (indexType == FunctionArgumentsConstants.RTREE_INDEX) {
-                                op.setPhysicalOperator(new RTreeSearchPOperator(dsi));
-                            } else {
+                            case RTREE: {
+                                op.setPhysicalOperator(new RTreeSearchPOperator(dsi, requiresBroadcast));
+                                break;
+                            }
+                            case WORD_INVIX: {
+                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+                                break;
+                            }
+                            case NGRAM_INVIX: {
+                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+                                break;
+                            }
+                            default: {
                                 throw new NotImplementedException(indexType + " indexes are not implemented.");
                             }
                         }
                     }
-                    if (notSet) {
-                        throw new IllegalStateException();
-                    }
                     break;
                 }
             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java
new file mode 100644
index 0000000..60e5a3e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java
@@ -0,0 +1,279 @@
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Looks for a select operator, containing a condition:
+ * 
+ * similarity-function GE/GT/LE/LE constant/variable
+ * 
+ * Rewrites the select condition (and possibly the assign expr) with the equivalent similarity-check function.
+ * 
+ */
+public class SimilarityCheckRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    	AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        // Look for select.
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        SelectOperator select = (SelectOperator) op;
+        Mutable<ILogicalExpression> condExpr = select.getCondition();
+        
+        // Gather assigns below this select.
+        List<AssignOperator> assigns = new ArrayList<AssignOperator>();
+        AbstractLogicalOperator childOp = (AbstractLogicalOperator) select.getInputs().get(0).getValue();
+        while (childOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+        	assigns.add((AssignOperator) childOp);
+        	childOp = (AbstractLogicalOperator) childOp.getInputs().get(0).getValue();
+        }
+        return replaceSelectConditionExprs(condExpr, assigns, context);
+    }
+
+    private boolean replaceSelectConditionExprs(Mutable<ILogicalExpression> expRef, List<AssignOperator> assigns, IOptimizationContext context) throws AlgebricksException {
+        ILogicalExpression expr = expRef.getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        // Recursively traverse conjuncts.
+        // TODO: Ignore disjuncts for now, because some replacements may be invalid.
+        // For example, if the result of the similarity function is used somewhere upstream,
+        // then we may still need the true similarity value even if the GE/GT/LE/LE comparison returns false.
+        if (funcIdent == AlgebricksBuiltinFunctions.AND) {
+            boolean found = true;
+            for (int i = 0; i < funcExpr.getArguments().size(); ++i) {
+                found = found && replaceSelectConditionExprs(funcExpr.getArguments().get(i), assigns, context);
+            }
+            return found;
+        }
+        
+        // Look for GE/GT/LE/LT.
+        if (funcIdent != AlgebricksBuiltinFunctions.GE && funcIdent != AlgebricksBuiltinFunctions.GT &&
+                funcIdent != AlgebricksBuiltinFunctions.LE && funcIdent != AlgebricksBuiltinFunctions.LT) {
+            return false;
+        }
+
+        // One arg should be a function call or a variable, the other a constant.
+        AsterixConstantValue constVal = null;
+        ILogicalExpression nonConstExpr = null;
+        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+        // Normalized GE/GT/LE/LT as if constant was on the right hand side.
+        FunctionIdentifier normFuncIdent = null;
+        // One of the args must be a constant.
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {                    	
+        	ConstantExpression constExpr = (ConstantExpression) arg1;
+            constVal = (AsterixConstantValue) constExpr.getValue();
+            nonConstExpr = arg2;
+            // Get func ident as if swapping lhs and rhs.            
+            normFuncIdent = getLhsAndRhsSwappedFuncIdent(funcIdent);
+        } else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            ConstantExpression constExpr = (ConstantExpression) arg2;
+            constVal = (AsterixConstantValue) constExpr.getValue();
+            nonConstExpr = arg1;
+            // Constant is already on rhs, so nothing to be done for normalizedFuncIdent.
+            normFuncIdent = funcIdent;
+        } else {
+            return false;
+        }
+        
+        // The other arg is a function call. We can directly replace the select condition with an equivalent similarity check expression.
+        if (nonConstExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+        	return replaceWithFunctionCallArg(expRef, normFuncIdent, constVal, (AbstractFunctionCallExpression) nonConstExpr);
+        }
+        // The other arg ist a variable. We may have to introduce an assign operator that assigns the result of a similarity-check function to a variable.
+        if (nonConstExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+        	return replaceWithVariableArg(expRef, normFuncIdent, constVal, (VariableReferenceExpression) nonConstExpr, assigns, context);
+        }
+        return false;
+    }
+    
+    private boolean replaceWithVariableArg(Mutable<ILogicalExpression> expRef, FunctionIdentifier normFuncIdent,
+    		AsterixConstantValue constVal, VariableReferenceExpression varRefExpr, List<AssignOperator> assigns, IOptimizationContext context) throws AlgebricksException {
+    	
+    	// Find variable in assigns to determine its originating function.    	
+    	LogicalVariable var = varRefExpr.getVariableReference();
+    	Mutable<ILogicalExpression> simFuncExprRef = null;
+    	ScalarFunctionCallExpression simCheckFuncExpr = null;
+    	AssignOperator matchingAssign = null;
+    	for (int i = 0; i < assigns.size(); i++) {
+    		AssignOperator assign = assigns.get(i);
+    		for (int j = 0; j < assign.getVariables().size(); j++) {
+    			// Check if variables match.
+    			if (var != assign.getVariables().get(j)) {
+    				continue;
+    			}
+    			// Check if corresponding expr is a function call.
+    			if (assign.getExpressions().get(j).getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+    				continue;
+    			}
+    			simFuncExprRef = assign.getExpressions().get(j);
+    			// Analyze function expression and get equivalent similarity check function.
+    			simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal, (AbstractFunctionCallExpression) simFuncExprRef.getValue());
+    			matchingAssign = assign;
+    			break;
+    		}
+    		if (simCheckFuncExpr != null) {
+    			break;
+    		}
+    	}
+    	
+    	// Only non-null if we found that varRefExpr refers to an optimizable similarity function call. 
+    	if (simCheckFuncExpr != null) {
+    		// Create a new assign under matchingAssign which assigns the result of our similarity-check function to a variable.
+    		LogicalVariable newVar = context.newVar();
+    		AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(simCheckFuncExpr));
+    		// Hook up inputs. 
+    		newAssign.getInputs().add(new MutableObject<ILogicalOperator>(matchingAssign.getInputs().get(0).getValue()));
+    		matchingAssign.getInputs().get(0).setValue(newAssign);    		
+    		
+    		// Replace select condition with a get-item on newVar.
+            List<Mutable<ILogicalExpression>> selectGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            // First arg is a variable reference expr on newVar.
+            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            // Second arg is the item index to be accessed, here 0.
+            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
+            ILogicalExpression selectGetItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), selectGetItemArgs);
+            // Replace the old similarity function call with the new getItemExpr.
+            expRef.setValue(selectGetItemExpr);
+    		
+            // Replace expr corresponding to original variable in the original assign with a get-item on newVar.
+            List<Mutable<ILogicalExpression>> assignGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            // First arg is a variable reference expr on newVar.
+            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            // Second arg is the item index to be accessed, here 1.
+            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
+            ILogicalExpression assignGetItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), assignGetItemArgs);
+            // Replace the original assign expr with the get-item expr.
+            simFuncExprRef.setValue(assignGetItemExpr);
+    		
+            context.computeAndSetTypeEnvironmentForOperator(newAssign);
+            context.computeAndSetTypeEnvironmentForOperator(matchingAssign);
+            
+    		return true;
+    	}
+    	
+    	return false;
+    }
+    
+    private boolean replaceWithFunctionCallArg(Mutable<ILogicalExpression> expRef, FunctionIdentifier normFuncIdent,
+    		AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
+    	// Analyze func expr to see if it is an optimizable similarity function.
+        ScalarFunctionCallExpression simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal, funcExpr); 
+        
+        // Replace the expr in the select condition.
+        if (simCheckFuncExpr != null) {
+            // Get item 0 from var.
+            List<Mutable<ILogicalExpression>> getItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            // First arg is the similarity-check function call.
+            getItemArgs.add(new MutableObject<ILogicalExpression>(simCheckFuncExpr));
+            // Second arg is the item index to be accessed.
+            getItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
+            ILogicalExpression getItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), getItemArgs);
+            // Replace the old similarity function call with the new getItemExpr.
+            expRef.setValue(getItemExpr);
+            return true;
+        }
+
+        return false;
+    }
+    
+    private ScalarFunctionCallExpression getSimilarityCheckExpr(FunctionIdentifier normFuncIdent,
+    		AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
+    	// Remember args from original similarity function to add them to the similarity-check function later.
+        ArrayList<Mutable<ILogicalExpression>> similarityArgs = null;
+        ScalarFunctionCallExpression simCheckFuncExpr = null; 
+        // Look for jaccard function call, and GE or GT.
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD) {
+            AFloat aFloat = (AFloat) constVal.getObject();
+            AFloat jaccThresh;
+            if (normFuncIdent == AlgebricksBuiltinFunctions.GE) {
+                jaccThresh = aFloat;
+            } else if (normFuncIdent == AlgebricksBuiltinFunctions.GT) {
+                float f = aFloat.getFloatValue() + Float.MIN_VALUE;
+                if (f > 1.0f) f = 1.0f;
+                jaccThresh = new AFloat(f);
+            } else {
+                return null;
+            }
+            similarityArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            similarityArgs.addAll(funcExpr.getArguments());
+            similarityArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(jaccThresh))));
+            simCheckFuncExpr = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK), similarityArgs);
+        }
+
+        // Look for edit-distance function call, and LE or LT.
+        if(funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE) {
+            AInt32 aInt = (AInt32) constVal.getObject();
+            AInt32 edThresh;
+            if (normFuncIdent == AlgebricksBuiltinFunctions.LE) {
+                edThresh = aInt;
+            } else if (normFuncIdent == AlgebricksBuiltinFunctions.LT) {
+                int ed = aInt.getIntegerValue() - 1;
+                if (ed < 0) ed = 0;
+                edThresh = new AInt32(ed);
+            } else {
+                return null;
+            }
+            similarityArgs = new ArrayList<Mutable<ILogicalExpression>>();
+            similarityArgs.addAll(funcExpr.getArguments());
+            similarityArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(edThresh))));
+            simCheckFuncExpr = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK), similarityArgs);
+        }
+        return simCheckFuncExpr;
+    }
+    
+    private FunctionIdentifier getLhsAndRhsSwappedFuncIdent(FunctionIdentifier oldFuncIdent) {
+        if (oldFuncIdent == AlgebricksBuiltinFunctions.GE) {
+            return AlgebricksBuiltinFunctions.LE;
+        }
+        if (oldFuncIdent == AlgebricksBuiltinFunctions.GT) {
+            return AlgebricksBuiltinFunctions.LT;
+        }
+        if (oldFuncIdent == AlgebricksBuiltinFunctions.LE) {
+            return AlgebricksBuiltinFunctions.GE;
+        }
+        if (oldFuncIdent == AlgebricksBuiltinFunctions.LT) {
+            return AlgebricksBuiltinFunctions.GT;
+        }
+        throw new IllegalStateException();
+    }
+    
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+}
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 fb0c656..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
@@ -6,13 +6,12 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.declared.ExternalFeedDataSource;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -78,8 +77,8 @@
 
                 AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                 AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
 
@@ -87,9 +86,8 @@
 
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
 
-                if (acdd.getDatasetType() == DatasetType.INTERNAL || acdd.getDatasetType() == DatasetType.FEED) {
-
-                    int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(acdd).size();
+                if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+                    int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
                     for (int i = 0; i < numPrimaryKeys; i++) {
                         v.add(context.newVar());
                     }
@@ -105,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.");
@@ -127,14 +125,14 @@
 
                 AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                 AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
+                Dataset dataset = metadata.findDataset(datasetName);
 
-                if (acdd == null) {
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
 
-                if (acdd.getDatasetType() != DatasetType.FEED) {
-                    throw new IllegalArgumentException("invalid dataset type:" + acdd.getDatasetType());
+                if (dataset.getDatasetType() != DatasetType.FEED) {
+                    throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
                 }
 
                 AqlSourceId asid = new AqlSourceId(metadata.getDataverseName(), datasetName);
@@ -149,7 +147,7 @@
 
                 v.add(unnest.getVariable());
 
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, acdd,
+                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
                         metadata));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
@@ -161,36 +159,22 @@
                 return true;
             }
         }
-        
 
         return false;
     }
 
-    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, AqlCompiledDatasetDecl acdd,
+    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
             AqlCompiledMetadataDeclarations metadata) throws AlgebricksException {
-
-        AqlCompiledFeedDatasetDetails feedDetails = (AqlCompiledFeedDatasetDetails) acdd.getAqlCompiledDatasetDetails();
-
         if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
             return null;
         }
-
-        String tName = acdd.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(tName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
-        AqlCompiledDatasetDecl dummySourceDecl = new AqlCompiledDatasetDecl(acdd.getName(), tName,
-                DatasetType.EXTERNAL, feedDetails);
-
-        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dummySourceDecl, itemType,
+        String tName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(tName);
+        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
                 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
new file mode 100644
index 0000000..b378ed7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -0,0 +1,334 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * 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) {
+        List<FunctionIdentifier> funcs = accessMethod.getOptimizableFunctions();
+        for (FunctionIdentifier funcIdent : funcs) {
+            List<IAccessMethod> l = accessMethods.get(funcIdent);
+            if (l == null) {
+                l = new ArrayList<IAccessMethod>();
+                accessMethods.put(funcIdent, l);
+            }
+            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 {
+        // 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();
+        Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
+        // Check applicability of indexes by access method type.
+        while (amIt.hasNext()) {
+            Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
+            AccessMethodAnalysisContext amCtx = entry.getValue();
+            // For the current access method type, map variables from the assign op to applicable indexes.
+            fillAllIndexExprs(varList, subTree, amCtx);
+        }
+    }
+
+    protected void pruneIndexCandidates(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+        Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
+        // Check applicability of indexes by access method type.
+        while (amIt.hasNext()) {
+            Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
+            AccessMethodAnalysisContext amCtx = entry.getValue();
+            pruneIndexCandidates(entry.getKey(), amCtx);
+            // Remove access methods for which there are definitely no applicable indexes.
+            if (amCtx.indexExprs.isEmpty()) {
+                amIt.remove();
+            }
+        }
+    }
+
+    /**
+     * 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) {
+        Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
+        while (amIt.hasNext()) {
+            Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
+            AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
+            Iterator<Map.Entry<Index, List<Integer>>> indexIt = analysisCtx.indexExprs.entrySet().iterator();
+            if (indexIt.hasNext()) {
+                Map.Entry<Index, List<Integer>> indexEntry = indexIt.next();
+                return new Pair<IAccessMethod, Index>(amEntry.getKey(), indexEntry.getKey());
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Removes irrelevant access methods candidates, based on whether the
+     * expressions in the query match those in the index. For example, some
+     * index may require all its expressions to be matched, and some indexes may
+     * 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();
+        while (it.hasNext()) {
+            Map.Entry<Index, List<Integer>> entry = it.next();
+            Index index = entry.getKey();
+            Iterator<Integer> exprsIter = entry.getValue().iterator();
+            boolean allUsed = true;
+            int lastFieldMatched = -1;
+            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+                String keyField = index.getKeyFieldNames().get(i);
+                boolean foundKeyField = false;
+                while (exprsIter.hasNext()) {
+                    Integer ix = exprsIter.next();
+                    IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(ix);
+                    // If expr is not optimizable by concrete index then remove expr and continue.
+                    if (!accessMethod.exprIsOptimizable(index, optFuncExpr)) {
+                        exprsIter.remove();
+                        continue;
+                    }
+                    // Check if any field name in the optFuncExpr matches.
+                    if (optFuncExpr.findFieldName(keyField) != -1) {
+                        foundKeyField = true;
+                        if (lastFieldMatched == i - 1) {
+                            lastFieldMatched = i;
+                        }
+                        break;
+                    }
+                }
+                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()) {
+                it.remove();
+                return;
+            }
+            // A prefix of the index exprs may have been matched.
+            if (lastFieldMatched < 0 && accessMethod.matchPrefixIndexExprs()) {
+                it.remove();
+                return;
+            }
+        }
+    }
+
+    /**
+     * 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) {
+        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);
+        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);
+            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) {
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        if (funcIdent == AlgebricksBuiltinFunctions.AND) {
+            return false;
+        }
+        // Retrieves the list of access methods that are relevant based on the funcIdent.
+        List<IAccessMethod> relevantAMs = getAccessMethods().get(funcIdent);
+        if (relevantAMs == null) {
+            return false;
+        }
+        boolean atLeastOneMatchFound = false;
+        // Place holder for a new analysis context in case we need one.
+        AccessMethodAnalysisContext newAnalysisCtx = new AccessMethodAnalysisContext();
+        for (IAccessMethod accessMethod : relevantAMs) {
+            AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(accessMethod);
+            // Use the current place holder.
+            if (analysisCtx == null) {
+                analysisCtx = newAnalysisCtx;
+            }
+            // Analyzes the funcExpr's arguments to see if the accessMethod is truly applicable.
+            boolean matchFound = accessMethod.analyzeFuncExprArgs(funcExpr, assigns, analysisCtx);
+            if (matchFound) {
+                // If we've used the current new context placeholder, replace it with a new one.
+                if (analysisCtx == newAnalysisCtx) {
+                    analyzedAMs.put(accessMethod, analysisCtx);
+                    newAnalysisCtx = new AccessMethodAnalysisContext();
+                }
+                atLeastOneMatchFound = true;
+            }
+        }
+        return atLeastOneMatchFound;
+    }
+
+    /**
+     * 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
+     */
+    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.
+        for (Index index : datasetIndexes) {
+            if (index.getKeyFieldNames().contains(fieldName)) {
+                indexCandidates.add(index);
+            }
+        }
+        // No index candidates for fieldName.
+        if (indexCandidates.isEmpty()) {
+            return false;
+        }
+        // Go through the candidates and fill indexExprs.
+        for (Index index : indexCandidates) {
+            analysisCtx.addIndexExpr(dataset, index, matchedFuncExprIndex);
+        }
+        return true;
+    }
+
+    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);
+                IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(optFuncExprIndex);
+                int funcVarIndex = optFuncExpr.findLogicalVar(var);
+                // No matching var in optFuncExpr.
+                if (funcVarIndex == -1) {
+                    continue;
+                }
+                // At this point we have matched the optimizable func expr at optFuncExprIndex to an assigned variable.
+                String fieldName = null;
+                if (!subTree.assigns.isEmpty()) {
+                    // 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);
+                    if (fieldName == null) {
+                        continue;
+                    }
+                } else {
+                    // We don't have an assign, only a datasource scan.
+                    // The last var. is the record itself, so skip it.
+                    if (varIndex >= varList.size() - 1) {
+                        break;
+                    }
+                    // The variable value is one of the partitioning fields.
+                    fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
+                }
+                // Set the fieldName in the corresponding matched function expression, and remember matching subtree.
+                optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
+                fillIndexExprs(fieldName, optFuncExprIndex, subTree.dataset, analysisCtx);
+            }
+        }
+    }
+
+    /**
+     * 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();
+        if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return null;
+        }
+        // Analyze the assign op to get the field name
+        // corresponding to the field being assigned at varIndex.
+        AbstractFunctionCallExpression assignFuncExpr = (AbstractFunctionCallExpression) assignExpr;
+        FunctionIdentifier assignFuncIdent = assignFuncExpr.getFunctionIdentifier();
+        if (assignFuncIdent == AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME) {
+            ILogicalExpression nameArg = assignFuncExpr.getArguments().get(1).getValue();
+            if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return null;
+            }
+            ConstantExpression constExpr = (ConstantExpression) nameArg;
+            return ((AString) ((AsterixConstantValue) constExpr.getValue()).getObject()).getStringValue();
+        } else if (assignFuncIdent == AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
+            ILogicalExpression idxArg = assignFuncExpr.getArguments().get(1).getValue();
+            if (idxArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return null;
+            }
+            ConstantExpression constExpr = (ConstantExpression) idxArg;
+            int fieldIndex = ((AInt32) ((AsterixConstantValue) constExpr.getValue()).getObject()).getIntegerValue();
+            return recordType.getFieldNames()[fieldIndex];
+        }
+        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
new file mode 100644
index 0000000..fc45063
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+
+/**
+ * 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);
+    }
+
+    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
new file mode 100644
index 0000000..af30163
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -0,0 +1,103 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+
+/**
+ * Helper class for reading and writing job-gen parameters for access methods to
+ * and from a list of function arguments, typically of an unnest-map.
+ */
+public class AccessMethodJobGenParams {
+    protected String indexName;
+    protected IndexType indexType;
+    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) {
+        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())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(datasetName)));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(retainInput)));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
+    }
+
+    public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
+        indexName = AccessMethodUtils.getStringConstant(funcArgs.get(0));
+        indexType = IndexType.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
+        datasetName = AccessMethodUtils.getStringConstant(funcArgs.get(2));
+        retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(3));
+        requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
+    }
+
+    public String getIndexName() {
+        return indexName;
+    }
+
+    public IndexType getIndexType() {
+        return indexType;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public boolean getRetainInput() {
+        return retainInput;
+    }
+
+    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()))));
+        funcArgs.add(numKeysRef);
+        for (LogicalVariable keyVar : varList) {
+            Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(
+                    new VariableReferenceExpression(keyVar));
+            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) {
+            for (int i = 0; i < numLowKeys; i++) {
+                LogicalVariable var = ((VariableReferenceExpression) funcArgs.get(index + 1 + i).getValue())
+                        .getVariableReference();
+                varList.add(var);
+            }
+        }
+        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
new file mode 100644
index 0000000..cd3712d
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -0,0 +1,275 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+
+/**
+ * Static helper functions for rewriting plans using indexes.
+ */
+public class AccessMethodUtils {
+    public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target) {
+        ARecordType recordType = (ARecordType) itemType;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            target.add(recordType.getFieldType(partitioningKey));
+        }
+        target.add(itemType);
+    }
+
+    public static ConstantExpression createStringConstant(String str) {
+        return new ConstantExpression(new AsterixConstantValue(new AString(str)));
+    }
+
+    public static ConstantExpression createInt32Constant(int i) {
+        return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
+    }
+
+    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 int getInt32Constant(Mutable<ILogicalExpression> expr) {
+        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();
+    }
+
+    public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
+            AccessMethodAnalysisContext analysisCtx) {
+        IAlgebricksConstantValue constFilterVal = null;
+        LogicalVariable fieldVar = null;
+        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+        // One of the args must be a constant, and the other arg must be a variable.
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+                && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            ConstantExpression constExpr = (ConstantExpression) arg1;
+            constFilterVal = constExpr.getValue();
+            VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
+            fieldVar = varExpr.getVariableReference();
+        } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+                && arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            ConstantExpression constExpr = (ConstantExpression) arg2;
+            constFilterVal = constExpr.getValue();
+            VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
+            fieldVar = varExpr.getVariableReference();
+        } else {
+            return false;
+        }
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
+        return true;
+    }
+
+    public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
+        switch (index.getIndexType()) {
+            case BTREE:
+            case WORD_INVIX:
+            case NGRAM_INVIX: {
+                return index.getKeyFieldNames().size();
+            }
+            case RTREE: {
+                Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames().get(0),
+                        recordType);
+                IAType keyType = keyPairType.first;
+                int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+                return numDimensions * 2;
+            }
+            default: {
+                throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
+            }
+        }
+    }
+
+    /**
+     * 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 {
+        if (!primaryKeysOnly) {
+            switch (index.getIndexType()) {
+                case BTREE:
+                case WORD_INVIX:
+                case NGRAM_INVIX: {
+                    for (String sk : index.getKeyFieldNames()) {
+                        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
+                        dest.add(keyPairType.first);
+                    }
+                    break;
+                }
+                case RTREE: {
+                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
+                            .get(0), recordType);
+                    IAType keyType = keyPairType.first;
+                    IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+                    int numKeys = getNumSecondaryKeys(index, recordType);
+                    for (int i = 0; i < numKeys; i++) {
+                        dest.add(nestedKeyType);
+                    }
+                    break;
+                }
+            }
+        }
+        // Primary keys.
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            dest.add(recordType.getFieldType(partitioningKey));
+        }
+    }
+
+    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;
+        for (int i = 0; i < numVars; i++) {
+            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>();
+        List<LogicalVariable> sourceVars = ((UnnestMapOperator) unnestMapOp).getVariables();
+        // Assumes the primary keys are located at the end.
+        int start = sourceVars.size() - numPrimaryKeys;
+        int stop = sourceVars.size();
+        for (int i = start; i < stop; i++) {
+            primaryKeyVars.add(sourceVars.get(i));
+        }
+        return primaryKeyVars;
+    }
+
+    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);
+        // Variables and types coming out of the secondary-index search. 
+        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);
+        // 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);
+        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));
+        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 {
+        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));
+                order.getOrderExpressions().add(
+                        new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+            }
+            // The secondary-index search feeds into the sort.
+            order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+            order.setExecutionMode(ExecutionMode.LOCAL);
+            context.computeAndSetTypeEnvironmentForOperator(order);
+        }
+        // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments. 
+        List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+        BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
+                dataset.getDatasetName(), retainInput, requiresBroadcast);
+        // Set low/high inclusive to true for a point lookup.
+        jobGenParams.setLowKeyInclusive(true);
+        jobGenParams.setHighKeyInclusive(true);
+        jobGenParams.setLowKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+        jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+        jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
+        // Variables and types coming out of the primary-index search.
+        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
+        List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+        // Append output variables/types generated by the primary-index search (not forwarded from input).
+        primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
+        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);
+        // 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);
+        // Fed by the order operator or the secondaryIndexUnnestOp.
+        if (sortPrimaryKeys) {
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+        } else {
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+        }
+        context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
+        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
new file mode 100644
index 0000000..414dca3
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -0,0 +1,411 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+
+/**
+ * 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
+    }
+
+    // TODO: There is some redundancy here, since these are listed in AlgebricksBuiltinFunctions as well.
+    private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+    static {
+        funcIdents.add(AlgebricksBuiltinFunctions.EQ);
+        funcIdents.add(AlgebricksBuiltinFunctions.LE);
+        funcIdents.add(AlgebricksBuiltinFunctions.GE);
+        funcIdents.add(AlgebricksBuiltinFunctions.LT);
+        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) {
+        return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+    }
+
+    @Override
+    public boolean matchAllIndexExprs() {
+        return true;
+    }
+
+    @Override
+    public boolean matchPrefixIndexExprs() {
+        // TODO: The BTree can support prefix searches. Enable this later and add tests.
+        return false;
+    }
+
+    @Override
+    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();
+        DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
+        Mutable<ILogicalOperator> assignRef = (subTree.assignRefs.isEmpty()) ? null : subTree.assignRefs.get(0);
+        AssignOperator assign = null;
+        if (assignRef != null) {
+            assign = (AssignOperator) assignRef.getValue();
+        }
+        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];
+        LimitType[] lowKeyLimits = new LimitType[numSecondaryKeys];
+        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.
+        // These func exprs will be removed from the select condition at the very end of this method.
+        Set<ILogicalExpression> replacedFuncExprs = new HashSet<ILogicalExpression>();
+        // TODO: For now we don't do any sophisticated analysis of the func exprs to come up with "the best" range predicate.
+        // If we can't figure out how to integrate a certain funcExpr into the current predicate, we just bail by setting this flag.
+        boolean couldntFigureOut = false;
+        boolean doneWithExprs = false;
+        // Go through the func exprs listed as optimizable by the chosen index, 
+        // and formulate a range predicate on the secondary-index keys.
+        for (Integer exprIndex : exprList) {
+            // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
+            IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex);
+            int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
+            if (keyPos < 0) {
+                throw new InternalError();
+            }
+            LimitType limit = getLimitType(optFuncExpr);
+            switch (limit) {
+                case EQUAL: {
+                    if (lowKeyLimits[keyPos] == null && highKeyLimits[keyPos] == null) {
+                        lowKeyLimits[keyPos] = highKeyLimits[keyPos] = limit;
+                        lowKeyInclusive[keyPos] = highKeyInclusive[keyPos] = true;
+                        lowKeyConstants[keyPos] = highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                    } else {
+                        couldntFigureOut = true;
+                    }
+                    // Mmmm, we would need an inference system here.
+                    doneWithExprs = true;
+                    break;
+                }
+                case HIGH_EXCLUSIVE: {
+                    if (highKeyLimits[keyPos] == null || (highKeyLimits[keyPos] != null && highKeyInclusive[keyPos])) {
+                        highKeyLimits[keyPos] = limit;
+                        highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        highKeyInclusive[keyPos] = false;
+                    } else {
+                        couldntFigureOut = true;
+                        doneWithExprs = true;
+                    }
+                    break;
+                }
+                case HIGH_INCLUSIVE: {
+                    if (highKeyLimits[keyPos] == null) {
+                        highKeyLimits[keyPos] = limit;
+                        highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        highKeyInclusive[keyPos] = true;
+                    } else {
+                        couldntFigureOut = true;
+                        doneWithExprs = true;
+                    }
+                    break;
+                }
+                case LOW_EXCLUSIVE: {
+                    if (lowKeyLimits[keyPos] == null || (lowKeyLimits[keyPos] != null && lowKeyInclusive[keyPos])) {
+                        lowKeyLimits[keyPos] = limit;
+                        lowKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        lowKeyInclusive[keyPos] = false;
+                    } else {
+                        couldntFigureOut = true;
+                        doneWithExprs = true;
+                    }
+                    break;
+                }
+                case LOW_INCLUSIVE: {
+                    if (lowKeyLimits[keyPos] == null) {
+                        lowKeyLimits[keyPos] = limit;
+                        lowKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        lowKeyInclusive[keyPos] = true;
+                    } else {
+                        couldntFigureOut = true;
+                        doneWithExprs = true;
+                    }
+                    break;
+                }
+                default: {
+                    throw new IllegalStateException();
+                }
+            }
+            if (!couldntFigureOut) {
+                // Remember to remove this funcExpr later.
+                replacedFuncExprs.add(matchedFuncExprs.get(exprIndex).getFuncExpr());
+            }
+            if (doneWithExprs) {
+                break;
+            }
+        }
+        if (couldntFigureOut) {
+            return false;
+        }
+
+        // Rule out the cases unsupported by the current btree search
+        // implementation.
+        for (int i = 1; i < numSecondaryKeys; i++) {
+            if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
+                return false;
+            }
+            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) {
+                return false;
+            }
+        }
+        if (lowKeyLimits[0] == null) {
+            lowKeyInclusive[0] = true;
+        }
+        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>();
+        // List of expressions for the assign.
+        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);
+
+        // 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);
+        } else {
+            List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+            AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
+            primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
+                    secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, false);
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
+        }
+
+        List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
+        getNewSelectExprs(select, replacedFuncExprs, remainingFuncExprs);
+        // Generate new select using the new condition.
+        if (!remainingFuncExprs.isEmpty()) {
+            ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
+            SelectOperator selectRest = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
+            if (assign != null) {
+                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+            } else {
+                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
+            }
+            selectRest.setExecutionMode(((AbstractLogicalOperator) selectRef.getValue()).getExecutionMode());
+            selectRef.setValue(selectRest);
+        } else {
+            primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
+            if (assign != null) {
+                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+                selectRef.setValue(assign);
+            } else {
+                selectRef.setValue(primaryIndexUnnestOp);
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+            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) {
+        if (keyLimits[0] == null) {
+            return 0;
+        }
+        int numKeys = keyLimits.length;
+        for (int i = 0; i < numKeys; i++) {
+            LogicalVariable keyVar = context.newVar();
+            keyVarList.add(keyVar);
+            keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(keyConstants[i])));
+        }
+        return numKeys;
+    }
+
+    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs,
+            List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+        remainingFuncExprs.clear();
+        if (replacedFuncExprs.isEmpty()) {
+            return;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) select.getCondition().getValue();
+        if (replacedFuncExprs.size() == 1) {
+            Iterator<ILogicalExpression> it = replacedFuncExprs.iterator();
+            if (!it.hasNext()) {
+                return;
+            }
+            if (funcExpr == it.next()) {
+                // There are no remaining function exprs.
+                return;
+            }
+        }
+        // The original select cond must be an AND. Check it just to be sure.
+        if (funcExpr.getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND) {
+            throw new IllegalStateException();
+        }
+        // Clean the conjuncts.
+        for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+            ILogicalExpression argExpr = arg.getValue();
+            if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                continue;
+            }
+            // If the function expression was not replaced by the new index
+            // plan, then add it to the list of remaining function expressions.
+            if (!replacedFuncExprs.contains(argExpr)) {
+                remainingFuncExprs.add(arg);
+            }
+        }
+    }
+
+    private <T> int indexOf(T value, List<T> coll) {
+        int i = 0;
+        for (T member : coll) {
+            if (member.equals(value)) {
+                return i;
+            }
+            i++;
+        }
+        return -1;
+    }
+
+    private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr) {
+        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr()
+                .getFunctionIdentifier());
+        LimitType limit = null;
+        switch (ck) {
+            case EQ: {
+                limit = LimitType.EQUAL;
+                break;
+            }
+            case GE: {
+                limit = constantIsOnLhs(optFuncExpr) ? LimitType.HIGH_INCLUSIVE : LimitType.LOW_INCLUSIVE;
+                break;
+            }
+            case GT: {
+                limit = constantIsOnLhs(optFuncExpr) ? LimitType.HIGH_EXCLUSIVE : LimitType.LOW_EXCLUSIVE;
+                break;
+            }
+            case LE: {
+                limit = constantIsOnLhs(optFuncExpr) ? LimitType.LOW_INCLUSIVE : LimitType.HIGH_INCLUSIVE;
+                break;
+            }
+            case LT: {
+                limit = constantIsOnLhs(optFuncExpr) ? LimitType.LOW_EXCLUSIVE : LimitType.HIGH_EXCLUSIVE;
+                break;
+            }
+            case NEQ: {
+                limit = null;
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+        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);
+    }
+
+    private ILogicalExpression createSelectCondition(List<Mutable<ILogicalExpression>> predList) {
+        if (predList.size() > 1) {
+            IFunctionInfo finfo = AsterixBuiltinFunctions.getAsterixFunctionInfo(AlgebricksBuiltinFunctions.AND);
+            return new ScalarFunctionCallExpression(finfo, predList);
+        }
+        return predList.get(0).getValue();
+    }
+
+    @Override
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        // No additional analysis required for BTrees.
+        return true;
+    }
+}
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
new file mode 100644
index 0000000..c377a34
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
@@ -0,0 +1,102 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+
+/**
+ * Helper class for reading and writing job-gen parameters for BTree access methods to
+ * and from a list of function arguments, typically of an unnest-map.
+ */
+public class BTreeJobGenParams extends AccessMethodJobGenParams {
+
+    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);
+    }
+
+    public void setLowKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
+        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);
+        writeVarList(highKeyVarList, funcArgs);
+        writeKeyInclusive(lowKeyInclusive, funcArgs);
+        writeKeyInclusive(highKeyInclusive, funcArgs);
+    }
+
+    public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
+        super.readFromFuncArgs(funcArgs);
+        int index = super.getNumParams();
+        lowKeyVarList = new ArrayList<LogicalVariable>();
+        highKeyVarList = new ArrayList<LogicalVariable>();
+        int nextIndex = readVarList(funcArgs, index, lowKeyVarList);
+        nextIndex = readVarList(funcArgs, nextIndex, highKeyVarList);
+        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;
+    }
+
+    public List<LogicalVariable> getHighKeyVarList() {
+        return highKeyVarList;
+    }
+
+    public boolean isLowKeyInclusive() {
+        return lowKeyInclusive;
+    }
+
+    public boolean isHighKeyInclusive() {
+        return highKeyInclusive;
+    }
+}
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
new file mode 100644
index 0000000..eadbc69
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
@@ -0,0 +1,77 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+
+/**
+ * Interface that an access method should implement to work with the rewrite
+ * rules to apply it for join and/or selection queries. This interface provides
+ * methods for analyzing a select/join condition, and for rewriting the plan
+ * 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
+     * var). We assume that the funcExpr has already been determined to be
+     * optimizable by this access method based on its function identifier. If
+     * funcExpr has been found to be optimizable, this method adds an
+     * OptimizableFunction to analysisCtx.matchedFuncExprs for further analysis.
+     * 
+     * @return true if funcExpr is optimizable by this access method, false
+     *         otherwise
+     */
+    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.
+     * 
+     * @return boolean
+     */
+    public boolean matchAllIndexExprs();
+
+    /**
+     * Indicates whether this index is applicable if only a prefix of the index
+     * expressions are matched.
+     * 
+     * @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;
+
+    /**
+     * 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;
+
+    /**
+     * Analyzes expr to see whether it is optimizable by the given concrete index.
+     */
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr);
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
new file mode 100644
index 0000000..dd91fc1
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+
+/**
+ * Describes a function expression that is optimizable by an access method.
+ * Provides convenient methods for accessing arguments (constants, variables)
+ * and metadata of such a function.
+ */
+public interface IOptimizableFuncExpr {
+    public AbstractFunctionCallExpression getFuncExpr();
+    public int getNumLogicalVars();
+    public int getNumConstantVals();
+    public LogicalVariable getLogicalVar(int index);
+    public void setFieldName(int index, String fieldName);
+    public String getFieldName(int index);
+    public void setOptimizableSubTree(int index, OptimizableOperatorSubTree subTree);
+    public OptimizableOperatorSubTree getOperatorSubTree(int index);
+    public IAlgebricksConstantValue getConstantVal(int index);
+    
+    public int findLogicalVar(LogicalVariable var);
+    public int findFieldName(String fieldName);
+}
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
new file mode 100644
index 0000000..e4555aa
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -0,0 +1,156 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+/**
+ * This rule optimizes a join with secondary indexes into an indexed nested-loop join.
+ * 
+ * Matches the following operator pattern:
+ * (join) <-- (select)? <-- (assign)+ <-- (datasource scan)
+ *        <-- (select)? <-- (assign)+ <-- (datasource scan)
+ * 
+ * Replaces the above pattern with the following simplified plan: 
+ * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest(index search)) <-- (assign) <-- (datasource scan)
+ * The sort is optional, and some access methods may choose not to sort.
+ * 
+ * Note that for some index-based optimizations we do not remove the triggering
+ * condition from the join, since the secondary index may only act as a filter, and the
+ * final verification must still be done with the original join condition.
+ * 
+ * The basic outline of this rule is: 
+ * 1. Match operator pattern. 
+ * 2. Analyze join condition to see if there are optimizable functions (delegated to IAccessMethods). 
+ * 3. Check metadata to see if there are applicable indexes. 
+ * 4. Choose an index to apply (for now only a single index will be chosen).
+ * 5. Rewrite plan using index (delegated to IAccessMethods).
+ * 
+ * TODO (Alex): Currently this rule requires a data scan on both inputs of the join. I should generalize the pattern 
+ * 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);
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        setMetadataDeclarations(context);
+
+        // Match operator pattern and initialize optimizable sub trees.
+        if (!matchesOperatorPattern(opRef, context)) {
+            return false;
+        }
+        // Analyze condition on those optimizable subtrees that have a datasource scan.
+        Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
+        boolean matchInLeftSubTree = false;
+        boolean matchInRightSubTree = false;
+        if (leftSubTree.hasDataSourceScan()) {
+            matchInLeftSubTree = analyzeCondition(joinCond, leftSubTree.assigns, analyzedAMs);
+        }
+        if (rightSubTree.hasDataSourceScan()) {
+            matchInRightSubTree = analyzeCondition(joinCond, rightSubTree.assigns, analyzedAMs);
+        }
+        if (!matchInLeftSubTree && !matchInRightSubTree) {
+            return false;
+        }
+
+        // Set dataset and type metadata.
+        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+        boolean checkLeftSubTreeMetadata = false;
+        boolean checkRightSubTreeMetadata = false;
+        if (matchInLeftSubTree) {
+            checkLeftSubTreeMetadata = leftSubTree.setDatasetAndTypeMetadata(metadataProvider);
+        }
+        if (matchInRightSubTree) {
+            checkRightSubTreeMetadata = rightSubTree.setDatasetAndTypeMetadata(metadataProvider);
+        }
+        if (!checkLeftSubTreeMetadata && !checkRightSubTreeMetadata) {
+            return false;
+        }
+        if (checkLeftSubTreeMetadata) {
+            fillSubTreeIndexExprs(leftSubTree, analyzedAMs);
+        }
+        if (checkRightSubTreeMetadata) {
+            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,
+                chosenIndex.second, analysisCtx, context);
+        if (res) {
+            OperatorPropertiesUtil.typeOpRec(opRef, context);
+        }
+        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();
+        if (context.checkIfInDontApplySet(this, op1)) {
+            return false;
+        }
+        if (op1.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+        // Set and analyze select.
+        joinRef = opRef;
+        join = (InnerJoinOperator) op1;
+        // Check that the select's condition is a function call.
+        ILogicalExpression condExpr = join.getCondition().getValue();
+        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        joinCond = (AbstractFunctionCallExpression) condExpr;
+        leftSubTree.initFromSubTree(op1.getInputs().get(0));
+        rightSubTree.initFromSubTree(op1.getInputs().get(1));
+        // One of the subtrees must have a datasource scan.
+        if (leftSubTree.hasDataSourceScan() || rightSubTree.hasDataSourceScan()) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods() {
+        return accessMethods;
+    }
+}
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
new file mode 100644
index 0000000..59b11fc
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -0,0 +1,137 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+/**
+ * This rule optimizes simple selections with secondary or primary indexes. The use of an
+ * index is expressed as an unnest-map over an index-search function which will be
+ * replaced with the appropriate embodiment during codegen.
+ * 
+ * Matches the following operator patterns:
+ * Standard secondary index pattern:
+ * There must be at least one assign, but there may be more, e.g., when matching similarity-jaccard-check().
+ * (select) <-- (assign)+ <-- (datasource scan)
+ * Primary index lookup pattern:
+ * Since no assign is necessary to get the primary key fields (they are already stored fields in the BTree tuples).
+ * (select) <-- (datasource scan)
+ * 
+ * Replaces the above patterns with this plan:
+ * (select) <-- (assign) <-- (btree search) <-- (sort) <-- (unnest-map(index search)) <-- (assign)
+ * The sort is optional, and some access methods implementations may choose not to sort.
+ * 
+ * Note that for some index-based optimizations we do not remove the triggering
+ * condition from the select, since the index may only acts as a filter, and the
+ * final verification must still be done with the original select condition.
+ * 
+ * The basic outline of this rule is: 
+ * 1. Match operator pattern. 
+ * 2. Analyze select condition to see if there are optimizable functions (delegated to IAccessMethods). 
+ * 3. Check metadata to see if there are applicable indexes. 
+ * 4. Choose an index to apply (for now only a single index will be chosen).
+ * 5. Rewrite plan using index (delegated to IAccessMethods).
+ * 
+ */
+public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMethodRule {
+
+    // 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();
+
+    // 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 {
+        setMetadataDeclarations(context);
+
+        // 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)) {
+            return false;
+        }
+
+        // Set dataset and type metadata.
+        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);
+        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();
+        if (context.checkIfInDontApplySet(this, op1)) {
+            return false;
+        }
+        if (op1.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        // Set and analyze select.
+        selectRef = opRef;
+        select = (SelectOperator) op1;
+        // Check that the select's condition is a function call.
+        ILogicalExpression condExpr = select.getCondition().getValue();
+        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        selectCond = (AbstractFunctionCallExpression) condExpr;
+        return subTree.initFromSubTree(op1.getInputs().get(0));
+    }
+
+    @Override
+    public Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods() {
+        return accessMethods;
+    }
+}
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
new file mode 100644
index 0000000..dbd92c0
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -0,0 +1,834 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.algebra.base.LogicalOperatorDeepCopyVisitor;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.dataflow.data.common.ListEditDistanceSearchModifierFactory;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.IACollection;
+import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.Counter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.EditDistanceSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.JaccardSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+/**
+ * Class for helping rewrite rules to choose and apply inverted indexes.
+ */
+public class InvertedIndexAccessMethod implements IAccessMethod {
+
+    // Enum describing the search modifier type. Used for passing info to jobgen.
+    public static enum SearchModifierType {
+        CONJUNCTIVE,
+        JACCARD,
+        EDIT_DISTANCE,
+        INVALID
+    }
+
+    private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+    static {
+        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>();
+    static {
+        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;
+    }
+
+    @Override
+    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) {
+        if (funcExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GET_ITEM) {
+            return false;
+        }
+        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+        // The second arg is the item index to be accessed. It must be a constant.
+        if (arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+        // 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) {
+            return false;
+        }
+        AbstractFunctionCallExpression matchedFuncExpr = null;
+        // The get-item arg is function call, directly check if it's optimizable.
+        if (arg1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            matchedFuncExpr = (AbstractFunctionCallExpression) arg1;
+        }
+        // The get-item arg is a variable. Search the assigns for its origination function.
+        int matchedAssignIndex = -1;
+        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++) {
+                AssignOperator assign = assigns.get(i);
+                List<LogicalVariable> assignVars = assign.getVariables();
+                List<Mutable<ILogicalExpression>> assignExprs = assign.getExpressions();
+                for (int j = 0; j < assignVars.size(); j++) {
+                    LogicalVariable var = assignVars.get(j);
+                    if (var != varRefExpr.getVariableReference()) {
+                        continue;
+                    }
+                    // We've matched the variable in the first assign. Now analyze the originating function.
+                    ILogicalExpression matchedExpr = assignExprs.get(j).getValue();
+                    if (matchedExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                        return false;
+                    }
+                    matchedAssignIndex = i;
+                    matchedFuncExpr = (AbstractFunctionCallExpression) matchedExpr;
+                    break;
+                }
+                // We've already found a match.
+                if (matchedFuncExpr != null) {
+                    break;
+                }
+            }
+        }
+        // Check that the matched function is optimizable by this access method.
+        if (!secondLevelFuncIdents.contains(matchedFuncExpr.getFunctionIdentifier())) {
+            return false;
+        }
+        boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns,
+                matchedAssignIndex, analysisCtx);
+        boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex,
+                analysisCtx);
+        if (selectMatchFound || joinMatchFound) {
+            return true;
+        }
+        return false;
+    }
+
+    private boolean analyzeJoinSimilarityCheckFuncExprArgs(AbstractFunctionCallExpression funcExpr,
+            List<AssignOperator> assigns, int matchedAssignIndex, AccessMethodAnalysisContext analysisCtx) {
+        // There should be exactly three arguments.
+        // The last function argument is assumed to be the similarity threshold.
+        IAlgebricksConstantValue constThreshVal = null;
+        ILogicalExpression arg3 = funcExpr.getArguments().get(2).getValue();
+        if (arg3.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+        constThreshVal = ((ConstantExpression) arg3).getValue();
+        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
+                || arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+        LogicalVariable fieldVar1 = getNonConstArgFieldVar(arg1, funcExpr, assigns, matchedAssignIndex);
+        if (fieldVar1 == null) {
+            return false;
+        }
+        LogicalVariable fieldVar2 = getNonConstArgFieldVar(arg2, funcExpr, assigns, matchedAssignIndex);
+        if (fieldVar2 == null) {
+            return false;
+        }
+        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.
+        // The last function argument is assumed to be the similarity threshold.
+        IAlgebricksConstantValue constThreshVal = null;
+        ILogicalExpression arg3 = funcExpr.getArguments().get(2).getValue();
+        if (arg3.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+        constThreshVal = ((ConstantExpression) arg3).getValue();
+        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+        // 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) {
+            constArg = arg1;
+            nonConstArg = arg2;
+        } else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
+                && arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            constArg = arg2;
+            nonConstArg = arg1;
+        } else {
+            return false;
+        }
+        ConstantExpression constExpr = (ConstantExpression) constArg;
+        IAlgebricksConstantValue constFilterVal = constExpr.getValue();
+        LogicalVariable fieldVar = getNonConstArgFieldVar(nonConstArg, funcExpr, assigns, matchedAssignIndex);
+        if (fieldVar == null) {
+            return false;
+        }
+        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) {
+        LogicalVariable fieldVar = null;
+        // Analyze nonConstArg depending on similarity function.
+        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) {
+                    return null;
+                }
+                // Find the variable that is being tokenized.
+                nonConstArg = nonConstFuncExpr.getArguments().get(0).getValue();
+            }
+            if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression varExpr = (VariableReferenceExpression) nonConstArg;
+                fieldVar = varExpr.getVariableReference();
+                // Find expr corresponding to var in assigns below.
+                for (int i = matchedAssignIndex + 1; i < assigns.size(); i++) {
+                    AssignOperator assign = assigns.get(i);
+                    boolean found = false;
+                    for (int j = 0; j < assign.getVariables().size(); j++) {
+                        if (fieldVar != assign.getVariables().get(j)) {
+                            continue;
+                        }
+                        ILogicalExpression childExpr = assign.getExpressions().get(j).getValue();
+                        if (childExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                            break;
+                        }
+                        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) {
+                            break;
+                        }
+                        // We expect the tokenizer's argument to be a variable, otherwise we cannot apply an index.
+                        ILogicalExpression tokArgExpr = childFuncExpr.getArguments().get(0).getValue();
+                        if (tokArgExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                            break;
+                        }
+                        // Pass the variable being tokenized to the optimizable func expr.
+                        VariableReferenceExpression tokArgVarExpr = (VariableReferenceExpression) tokArgExpr;
+                        fieldVar = tokArgVarExpr.getVariableReference();
+                        found = true;
+                        break;
+                    }
+                    if (found) {
+                        break;
+                    }
+                }
+            }
+        }
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
+            if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();
+            }
+        }
+        return fieldVar;
+    }
+
+    @Override
+    public boolean matchAllIndexExprs() {
+        return true;
+    }
+
+    @Override
+    public boolean matchPrefixIndexExprs() {
+        return false;
+    }
+
+    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.
+        // List of variables for the assign.
+        ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
+        // probeSubTree is null if we are dealing with a selection query, and non-null for join queries.
+        if (probeSubTree == null) {
+            // List of expressions for the assign.
+            ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+            // 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);
+            // 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());
+        } else {
+            // We are optimizing a join. Add the input variable to the secondaryIndexFuncArgs.
+            LogicalVariable inputSearchVariable = getInputSearchVar(optFuncExpr, indexSubTree);
+            keyVarList.add(inputSearchVariable);
+            inputOp = (AbstractLogicalOperator) probeSubTree.root;
+        }
+        jobGenParams.setKeyVarList(keyVarList);
+        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);
+        return primaryIndexUnnestOp;
+    }
+
+    /**
+     * Returns the variable which acts as the input search key to a secondary
+     * index that optimizes optFuncExpr by replacing rewriting indexSubTree
+     * (which is the original subtree that will be replaced by the index plan).
+     */
+    private LogicalVariable getInputSearchVar(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree) {
+        if (optFuncExpr.getOperatorSubTree(0) == indexSubTree) {
+            // If the index is on a dataset in subtree 0, then subtree 1 will feed.
+            return optFuncExpr.getLogicalVar(1);
+        } else {
+            // If the index is on a dataset in subtree 1, then subtree 0 will feed.
+            return optFuncExpr.getLogicalVar(0);
+        }
+    }
+
+    @Override
+    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);
+        // 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 {
+        // 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.
+        OptimizableOperatorSubTree indexSubTree = null;
+        OptimizableOperatorSubTree probeSubTree = null;
+        if (dataset.getDatasetName().equals(leftSubTree.dataset.getDatasetName())) {
+            indexSubTree = leftSubTree;
+            probeSubTree = rightSubTree;
+        } else if (dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+            indexSubTree = rightSubTree;
+            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);
+            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);
+        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);
+        }
+        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.
+        List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+        int firstExprIndex = indexExprs.get(0);
+        return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+    }
+
+    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);
+
+        // 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);
+        Mutable<ILogicalOperator> isFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
+        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);
+        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. 
+        InnerJoinOperator joinOp = (InnerJoinOperator) joinRef.getValue();
+        // Substitute vars in the join condition due to copying of the scanSubTree.
+        List<LogicalVariable> joinCondUsedVars = new ArrayList<LogicalVariable>();
+        VariableUtilities.getUsedVariables(joinOp, joinCondUsedVars);
+        for (int i = 0; i < joinCondUsedVars.size(); i++) {
+            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 {
+        // 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)));
+                // 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())));
+                // 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);
+                break;
+            }
+            case UNORDEREDLIST:
+            case ORDEREDLIST: {
+                List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
+                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);
+                break;
+            }
+            default: {
+            }
+        }
+        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));
+        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 {
+        // If we have two variables in the optFunxExpr, then we are optimizing a join.
+        IAType type = null;
+        ATypeTag typeTag = null;
+        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));
+            } 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));
+            }
+            typeTag = type.getTypeTag();
+        } 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();
+            type = obj.getType();
+            typeTag = type.getTypeTag();
+            if (typeTag != ATypeTag.ORDEREDLIST && typeTag != ATypeTag.STRING) {
+                throw new AlgebricksException("Only ordered lists and string types supported.");
+            }
+        }
+        jobGenParams.setSearchKeyType(typeTag);
+    }
+
+    private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
+            jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
+            jobGenParams.setSimilarityThreshold(new AsterixConstantValue(ANull.NULL));
+        }
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
+            jobGenParams.setSearchModifierType(SearchModifierType.JACCARD);
+            // Add the similarity threshold which, by convention, is the last constant value.
+            jobGenParams.setSimilarityThreshold(optFuncExpr.getConstantVal(optFuncExpr.getNumConstantVals() - 1));
+        }
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
+            jobGenParams.setSearchModifierType(SearchModifierType.EDIT_DISTANCE);
+            // Add the similarity threshold which, by convention, is the last constant value.
+            jobGenParams.setSimilarityThreshold(optFuncExpr.getConstantVal(optFuncExpr.getNumConstantVals() - 1));
+        }
+    }
+
+    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))));
+        return;
+    }
+
+    @Override
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
+            // Must be for a join query.
+            if (optFuncExpr.getNumConstantVals() == 1) {
+                return true;
+            }
+            // Check for panic in selection query.
+            // TODO: Panic also depends on prePost which is currently hardcoded to be true.
+            AsterixConstantValue listOrStrConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
+            AsterixConstantValue intConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(1);
+            IAObject listOrStrObj = listOrStrConstVal.getObject();
+            IAObject intObj = intConstVal.getObject();
+            AInt32 edThresh = (AInt32) intObj;
+            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;
+                // Compute merge threshold.
+                mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
+                        - edThresh.getIntegerValue() * index.getGramLength();
+            }
+            // 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;
+                // Compute merge threshold.
+                mergeThreshold = alist.size() - edThresh.getIntegerValue();
+            }
+            if (mergeThreshold <= 0) {
+                // We cannot use index to optimize expr.
+                return false;
+            }
+            return true;
+        }
+        // TODO: We need more checking: gram length, prePost, etc.
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
+            // Check the tokenization function of the non-constant func arg to see if it fits the concrete index type.
+            ILogicalExpression arg1 = optFuncExpr.getFuncExpr().getArguments().get(0).getValue();
+            ILogicalExpression arg2 = optFuncExpr.getFuncExpr().getArguments().get(1).getValue();
+            ILogicalExpression nonConstArg = null;
+            if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                nonConstArg = arg1;
+            } else {
+                nonConstArg = arg2;
+            }
+            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) {
+                    return true;
+                }
+                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
+                        && index.getIndexType() == IndexType.NGRAM_INVIX) {
+                    return true;
+                }
+            }
+            // The non-constant arg is not a function call. Perhaps a variable?
+            // We must have already verified during our analysis of the select condition, that this variable
+            // refers to a list, or to a tokenization function.
+            if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                return true;
+            }
+        }
+        // We can only optimize contains with ngram indexes.
+        if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS
+                && index.getIndexType() == IndexType.NGRAM_INVIX) {
+            // Check that the constant search string has at least gramLength characters.
+            AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
+            IAObject strObj = strConstVal.getObject();
+            if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
+                AString astr = (AString) strObj;
+                if (astr.getStringValue().length() >= index.getGramLength()) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
+        IAType type = keyType;
+        ATypeTag typeTag = keyType.getTypeTag();
+        // Extract item type from list.
+        if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
+            AbstractCollectionType listType = (AbstractCollectionType) keyType;
+            if (!listType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = listType.getItemType();
+        }
+        // Ignore case for string types.
+        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
+    }
+
+    public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
+        IAType type = keyType;
+        ATypeTag typeTag = keyType.getTypeTag();
+        // Extract item type from list.
+        if (typeTag == ATypeTag.UNORDEREDLIST) {
+            AUnorderedListType ulistType = (AUnorderedListType) keyType;
+            if (!ulistType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = ulistType.getItemType();
+        }
+        if (typeTag == ATypeTag.ORDEREDLIST) {
+            AOrderedListType olistType = (AOrderedListType) keyType;
+            if (!olistType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = olistType.getItemType();
+        }
+        return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
+    }
+
+    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
+            ATypeTag searchKeyType, Index index) throws AlgebricksException {
+        switch (index.getIndexType()) {
+            case WORD_INVIX: {
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(searchKeyType, false);
+            }
+            case NGRAM_INVIX: {
+                // Make sure not to use pre- and postfixing for conjunctive searches.
+                boolean prePost = (searchModifierType == SearchModifierType.CONJUNCTIVE) ? false : true;
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(searchKeyType,
+                        index.getGramLength(), prePost, false);
+            }
+            default: {
+                throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getIndexType() + "'.");
+            }
+        }
+    }
+
+    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
+            int gramLength) throws AlgebricksException {
+        switch (indexType) {
+            case WORD_INVIX: {
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
+            }
+            case NGRAM_INVIX: {
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
+                        false);
+            }
+            default: {
+                throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
+            }
+        }
+    }
+
+    public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
+            IAObject simThresh, Index index) throws AlgebricksException {
+        switch (searchModifierType) {
+            case CONJUNCTIVE: {
+                return new ConjunctiveSearchModifierFactory();
+            }
+            case JACCARD: {
+                float jaccThresh = ((AFloat) simThresh).getFloatValue();
+                return new JaccardSearchModifierFactory(jaccThresh);
+            }
+            case EDIT_DISTANCE: {
+                int edThresh = ((AInt32) simThresh).getIntegerValue();
+                switch (index.getIndexType()) {
+                    case NGRAM_INVIX: {
+                        // Edit distance on strings, filtered with overlapping grams.
+                        return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
+                    }
+                    case WORD_INVIX: {
+                        // Edit distance on two lists. The list-elements are non-overlapping.
+                        return new ListEditDistanceSearchModifierFactory(edThresh);
+                    }
+                    default: {
+                        throw new AlgebricksException("Incompatible search modifier '" + searchModifierType
+                                + "' for index type '" + index.getIndexType() + "'");
+                    }
+                }
+            }
+            default: {
+                throw new AlgebricksException("Unknown search modifier type '" + 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
new file mode 100644
index 0000000..530606e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -0,0 +1,111 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+
+/**
+ * Helper class for reading and writing job-gen parameters for RTree access methods to
+ * 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) {
+        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())));
+        // Write similarity threshold.
+        funcArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(similarityThreshold)));
+        // Write search key type.
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType
+                .ordinal())));
+        // Write key var list.
+        writeVarList(keyVarList, funcArgs);
+        // Write non-key var list.
+        if (nonKeyVarList != null) {
+            writeVarList(nonKeyVarList, funcArgs);
+        }
+    }
+
+    public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
+        super.readFromFuncArgs(funcArgs);
+        int index = super.getNumParams();
+        // Read search modifier type.
+        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());
+        // Read type of search key.
+        int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + 2));
+        searchKeyType = ATypeTag.values()[typeTagOrdinal];
+        // Read key var list.
+        keyVarList = new ArrayList<LogicalVariable>();
+        readVarList(funcArgs, index + 3, keyVarList);
+        // TODO: We could possibly simplify things if we did read the non-key var list here.
+        // 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;
+    }
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java
new file mode 100644
index 0000000..13e515a
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java
@@ -0,0 +1,99 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+
+/**
+ * General-purpose implementation of IOptimizableFuncExpr that supports any
+ * number of constant args, variable args and field names.
+ */
+public class OptimizableFuncExpr implements IOptimizableFuncExpr {
+	protected final AbstractFunctionCallExpression funcExpr;
+    protected final LogicalVariable[] logicalVars;
+    protected final String[] fieldNames;
+    protected final OptimizableOperatorSubTree[] subTrees;
+    protected final IAlgebricksConstantValue[] constantVals;
+    
+    public OptimizableFuncExpr(AbstractFunctionCallExpression funcExpr, LogicalVariable[] logicalVars, IAlgebricksConstantValue[] constantVals) {
+    	this.funcExpr = funcExpr;
+    	this.logicalVars = logicalVars;
+    	this.constantVals = constantVals;
+    	this.fieldNames = new String[logicalVars.length];
+    	this.subTrees = new OptimizableOperatorSubTree[logicalVars.length];
+    }
+    
+    // Special, more convenient c'tor for simple binary functions.
+    public OptimizableFuncExpr(AbstractFunctionCallExpression funcExpr, LogicalVariable logicalVar, IAlgebricksConstantValue constantVal) {
+    	this.funcExpr = funcExpr;
+    	this.logicalVars = new LogicalVariable[] { logicalVar };
+    	this.constantVals = new IAlgebricksConstantValue[] { constantVal };
+    	this.fieldNames = new String[logicalVars.length];
+    	this.subTrees = new OptimizableOperatorSubTree[logicalVars.length];
+    }
+    
+	@Override
+	public AbstractFunctionCallExpression getFuncExpr() {
+		return funcExpr;
+	}
+	
+	@Override
+	public int getNumLogicalVars() {
+		return logicalVars.length;
+	}
+	
+	@Override
+	public int getNumConstantVals() {
+		return constantVals.length;
+	}
+	
+	@Override	
+	public LogicalVariable getLogicalVar(int index) {
+		return logicalVars[index];
+	}
+	
+	@Override
+	public void setFieldName(int index, String fieldName) {
+		fieldNames[index] = fieldName;
+	}
+	
+	@Override
+	public String getFieldName(int index) {
+		return fieldNames[index];
+	}
+	
+	@Override
+	public IAlgebricksConstantValue getConstantVal(int index) {
+		return constantVals[index];
+	}
+
+    @Override
+    public int findLogicalVar(LogicalVariable var) {
+        for (int i = 0; i < logicalVars.length; i++) {
+            if (var == logicalVars[i]) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public int findFieldName(String fieldName) {
+        for (int i = 0; i < fieldNames.length; i++) {
+            if (fieldName.equals(fieldNames[i])) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public void setOptimizableSubTree(int index, OptimizableOperatorSubTree subTree) {
+        subTrees[index] = subTree;
+    }
+
+    @Override
+    public OptimizableOperatorSubTree getOperatorSubTree(int index) {
+        return subTrees[index];
+    }
+}
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
new file mode 100644
index 0000000..80f8cc1
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -0,0 +1,114 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.optimizer.base.AnalysisUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+
+/**
+ * Operator subtree that matches the following patterns, and provides convenient access to its nodes:
+ * (select)? <-- (assign)+ <-- (datasource scan)
+ * and
+ * (select)? <-- (datasource scan)
+ */
+public class OptimizableOperatorSubTree {
+    public ILogicalOperator root;
+    public Mutable<ILogicalOperator> rootRef;
+    public final List<Mutable<ILogicalOperator>> assignRefs = new ArrayList<Mutable<ILogicalOperator>>();
+    public final List<AssignOperator> assigns = new ArrayList<AssignOperator>();
+    public Mutable<ILogicalOperator> dataSourceScanRef = null;
+    public DataSourceScanOperator dataSourceScan = null;
+    // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
+    public Dataset dataset = null;
+    public ARecordType recordType = null;
+
+    public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
+        rootRef = subTreeOpRef;
+        root = subTreeOpRef.getValue();
+        // Examine the op's children to match the expected patterns.
+        AbstractLogicalOperator subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+        // Skip select operator.
+        if (subTreeOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+            subTreeOpRef = subTreeOp.getInputs().get(0);
+            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+        }
+        // Check primary-index pattern.
+        if (subTreeOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            // Pattern may still match if we are looking for primary index matches as well.
+            if (subTreeOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                dataSourceScanRef = subTreeOpRef;
+                dataSourceScan = (DataSourceScanOperator) subTreeOp;
+                return true;
+            }
+            return false;
+        }
+        // Match (assign)+.
+        do {
+            assignRefs.add(subTreeOpRef);
+            assigns.add((AssignOperator) subTreeOp);
+            subTreeOpRef = subTreeOp.getInputs().get(0);
+            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
+        } while (subTreeOp.getOperatorTag() == LogicalOperatorTag.ASSIGN);
+        // Set to last valid assigns.
+        subTreeOpRef = assignRefs.get(assignRefs.size() - 1);
+        subTreeOp = assigns.get(assigns.size() - 1);
+        // Match datasource scan.
+        Mutable<ILogicalOperator> opRef3 = subTreeOp.getInputs().get(0);
+        AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
+        if (op3.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+            return false;
+        }
+        dataSourceScanRef = opRef3;
+        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;
+        }
+        // Find the dataset corresponding to the datasource scan in the metadata.
+        String datasetName = AnalysisUtil.getDatasetName(dataSourceScan);
+        if (datasetName == null) {
+            return false;
+        }
+        AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
+        dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("No metadata for dataset " + datasetName);
+        }
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+            return false;
+        }
+        // Get the record type for that dataset.
+        IAType itemType = metadata.findType(dataset.getItemTypeName());
+        if (itemType.getTypeTag() != ATypeTag.RECORD) {
+            return false;
+        }
+        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
new file mode 100644
index 0000000..dfd3ff7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -0,0 +1,147 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+
+/**
+ * Class for helping rewrite rules to choose and apply RTree indexes.
+ */
+public class RTreeAccessMethod implements IAccessMethod {
+
+    private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
+    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) {
+        return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+    }
+
+    @Override
+    public boolean matchAllIndexExprs() {
+        return true;
+    }
+
+    @Override
+    public boolean matchPrefixIndexExprs() {
+        return false;
+    }
+
+    @Override
+    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.
+        // Pick the first expr optimizable by this index.
+        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,
+                dataset.getDatasetName(), false, false);
+        // A spatial object is serialized in the constant of the func expr we are optimizing.
+        // The R-Tree expects as input an MBR represented with 1 field per dimension. 
+        // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the R-Tree expects them).
+        // List of variables for the assign.
+        ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
+        // List of expressions for the assign.
+        ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+        for (int i = 0; i < numSecondaryKeys; i++) {
+            // The create MBR function "extracts" one field of an MBR around the given spatial object.
+            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))));
+            // The number of dimensions.
+            createMBR.getArguments().add(
+                    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)))));
+            // 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>(createMBR));
+        }
+        jobGenParams.setKeyVarList(keyVarList);
+
+        // Assign operator that "extracts" the MBR fields from the func-expr constant into a tuple.
+        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,
+                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);
+        // 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 {
+        // TODO Implement this.
+        return false;
+    }
+
+    @Override
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        // No additional analysis required.
+        return true;
+    }
+}
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
new file mode 100644
index 0000000..b3153f9
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+/**
+ * Helper class for reading and writing job-gen parameters for RTree access methods to
+ * 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) {
+        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 b1970ac..35fb3ae 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
@@ -80,8 +80,6 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
@@ -89,6 +87,8 @@
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
 import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -154,1258 +154,1507 @@
  * source for the current subtree.
  */
 
-public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator implements
-        IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator
+		implements
+		IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-    private final MetadataTransactionContext mdTxnCtx;
-    private final long txnId;
-    private TranslationContext context;
-    private String outputDatasetName;
-    private Statement.Kind dmlKind;
-    private static AtomicLong outputFileID = new AtomicLong(0);
-    private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
+	private final MetadataTransactionContext mdTxnCtx;
+	private final long txnId;
+	private TranslationContext context;
+	private String outputDatasetName;
+	private Statement.Kind dmlKind;
+	private static AtomicLong outputFileID = new AtomicLong(0);
+	private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
 
-    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-    public AqlExpressionToPlanTranslator(long txnId, MetadataTransactionContext mdTxnCtx, int currentVarCounter,
-            String outputDatasetName, Statement.Kind dmlKind) {
-        this.mdTxnCtx = mdTxnCtx;
-        this.txnId = txnId;
-        this.context = new TranslationContext(new Counter(currentVarCounter));
-        this.outputDatasetName = outputDatasetName;
-        this.dmlKind = dmlKind;
-    }
+	public AqlExpressionToPlanTranslator(long txnId,
+			MetadataTransactionContext mdTxnCtx, int currentVarCounter,
+			String outputDatasetName, Statement.Kind dmlKind) {
+		this.mdTxnCtx = mdTxnCtx;
+		this.txnId = txnId;
+		this.context = new TranslationContext(new Counter(currentVarCounter));
+		this.outputDatasetName = outputDatasetName;
+		this.dmlKind = dmlKind;
+	}
 
-    public int getVarCounter() {
-        return context.getVarCounter();
-    }
+	public int getVarCounter() {
+		return context.getVarCounter();
+	}
 
-    public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
-            throws AlgebricksException, AsterixException {
-        if (expr == null) {
-            return null;
-        }
-        if (compiledDeclarations == null) {
-            compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
-        }
-        if (!compiledDeclarations.isConnectedToDataverse())
-            compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
-        IDataFormat format = compiledDeclarations.getFormat();
-        if (format == null) {
-            throw new AlgebricksException("Data format has not been set.");
-        }
-        format.registerRuntimeFunctions();
-        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
-                new EmptyTupleSourceOperator()));
+	public ILogicalPlanAndMetadata translate(Query expr,
+			AqlCompiledMetadataDeclarations compiledDeclarations)
+			throws AlgebricksException, AsterixException {
+		if (expr == null) {
+			return null;
+		}
+		if (compiledDeclarations == null) {
+			compiledDeclarations = compileMetadata(mdTxnCtx,
+					expr.getPrologDeclList(), true);
+		}
+		if (!compiledDeclarations.isConnectedToDataverse())
+			compiledDeclarations.connectToDataverse(compiledDeclarations
+					.getDataverseName());
+		IDataFormat format = compiledDeclarations.getFormat();
+		if (format == null) {
+			throw new AlgebricksException("Data format has not been set.");
+		}
+		format.registerRuntimeFunctions();
+		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+				new MutableObject<ILogicalOperator>(
+						new EmptyTupleSourceOperator()));
 
-        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-        boolean isTransactionalWrite = false;
-        ILogicalOperator topOp = p.first;
-        ProjectOperator project = (ProjectOperator) topOp;
-        LogicalVariable resVar = project.getVariables().get(0);
-        if (outputDatasetName == null) {
-            FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
-            if (outputFileSplit == null) {
-                outputFileSplit = getDefaultOutputFileLocation();
-            }
-            compiledDeclarations.setOutputFile(outputFileSplit);
-            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
-            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
-            FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
-            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-            topOp = new WriteOperator(writeExprList, sink);
-            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-        } else {
-            String dataVerseName = compiledDeclarations.getDataverseName();
-            AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
-            if (adecl == null) {
-                throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
-            }
+		boolean isTransactionalWrite = false;
+		ILogicalOperator topOp = p.first;
+		ProjectOperator project = (ProjectOperator) topOp;
+		LogicalVariable resVar = project.getVariables().get(0);
+		if (outputDatasetName == null) {
+			FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
+			if (outputFileSplit == null) {
+				outputFileSplit = getDefaultOutputFileLocation();
+			}
+			compiledDeclarations.setOutputFile(outputFileSplit);
+			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			writeExprList.add(new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(resVar)));
+			FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
+			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+			topOp = new WriteOperator(writeExprList, sink);
+			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+		} else {
+			String dataVerseName = compiledDeclarations.getDataverseName();
+			Dataset dataset = compiledDeclarations
+					.findDataset(outputDatasetName);
+			if (dataset == null) {
+				throw new AlgebricksException("Cannot find dataset "
+						+ outputDatasetName);
+			}
 
-            AqlSourceId sourceId = new AqlSourceId(dataVerseName, outputDatasetName);
-            String itemTypeName = adecl.getItemTypeName();
-            IAType itemType = compiledDeclarations.findType(itemTypeName);
-            AqlDataSource dataSource = new AqlDataSource(sourceId, adecl, itemType);
+			AqlSourceId sourceId = new AqlSourceId(dataVerseName,
+					outputDatasetName);
+			String itemTypeName = dataset.getItemTypeName();
+			IAType itemType = compiledDeclarations.findType(itemTypeName);
+			AqlDataSource dataSource = new AqlDataSource(sourceId, dataset,
+					itemType);
+			if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+				throw new AlgebricksException(
+						"Cannot write output to an external dataset.");
+			}
+			ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+			ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+			List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
 
-            if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
-                throw new AlgebricksException("Cannot write output to an external dataset.");
-            }
-            ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-            ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
-            List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
+			List<String> partitionKeys = DatasetUtils
+					.getPartitioningKeys(dataset);
+			for (String keyFieldName : partitionKeys) {
+				IFunctionInfo finfoAccess = AsterixBuiltinFunctions
+						.getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
+				@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)))));
+				f.substituteVar(METADATA_DUMMY_VAR, resVar);
+				exprs.add(new MutableObject<ILogicalExpression>(f));
+				LogicalVariable v = context.newVar();
+				vars.add(v);
+				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(v)));
+			}
+			AssignOperator assign = new AssignOperator(vars, exprs);
+			assign.getInputs()
+					.add(new MutableObject<ILogicalOperator>(project));
 
-            AqlCompiledInternalDatasetDetails datasetDetails = (AqlCompiledInternalDatasetDetails) adecl
-                    .getAqlCompiledDatasetDetails();
-            List<String> partitionKeys = datasetDetails.getPartitioningExprs();
-            for (String keyFieldName : partitionKeys) {
-                IFunctionInfo finfoAccess = AsterixBuiltinFunctions
-                        .getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
-                @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)))));
-                f.substituteVar(METADATA_DUMMY_VAR, resVar);
-                exprs.add(new MutableObject<ILogicalExpression>(f));
-                LogicalVariable v = context.newVar();
-                vars.add(v);
-                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            AssignOperator assign = new AssignOperator(vars, exprs);
-            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+			Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(resVar));
+			ILogicalOperator load = null;
 
-            Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                    resVar));
-            ILogicalOperator load = null;
+			switch (dmlKind) {
+			case WRITE_FROM_QUERY_RESULT: {
+				load = new WriteResultOperator(dataSource, varRef,
+						varRefsForLoading);
+				load.getInputs().add(
+						new MutableObject<ILogicalOperator>(assign));
+				break;
+			}
+			case INSERT: {
+				ILogicalOperator insertOp = new InsertDeleteOperator(
+						dataSource, varRef, varRefsForLoading,
+						InsertDeleteOperator.Kind.INSERT);
+				insertOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(assign));
+				load = new SinkOperator();
+				load.getInputs().add(
+						new MutableObject<ILogicalOperator>(insertOp));
+				isTransactionalWrite = true;
+				break;
+			}
+			case DELETE: {
+				ILogicalOperator deleteOp = new InsertDeleteOperator(
+						dataSource, varRef, varRefsForLoading,
+						InsertDeleteOperator.Kind.DELETE);
+				deleteOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(assign));
+				load = new SinkOperator();
+				load.getInputs().add(
+						new MutableObject<ILogicalOperator>(deleteOp));
+				isTransactionalWrite = true;
+				break;
+			}
+			case BEGIN_FEED: {
+				ILogicalOperator insertOp = new InsertDeleteOperator(
+						dataSource, varRef, varRefsForLoading,
+						InsertDeleteOperator.Kind.INSERT);
+				insertOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(assign));
+				load = new SinkOperator();
+				load.getInputs().add(
+						new MutableObject<ILogicalOperator>(insertOp));
+				isTransactionalWrite = false;
+				break;
+			}
+			}
+			topOp = load;
+		}
 
-            switch (dmlKind) {
-                case WRITE_FROM_QUERY_RESULT: {
-                    load = new WriteResultOperator(dataSource, varRef, varRefsForLoading);
-                    load.getInputs().add(new MutableObject<ILogicalOperator>(assign));
-                    break;
-                }
-                case INSERT: {
-                    ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.INSERT);
-                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
-                    load = new SinkOperator();
-                    load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
-                    isTransactionalWrite = true;
-                    break;
-                }
-                case DELETE: {
-                    ILogicalOperator deleteOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.DELETE);
-                    deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
-                    load = new SinkOperator();
-                    load.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
-                    isTransactionalWrite = true;
-                    break;
-                }
-                case BEGIN_FEED: {
-                    ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.INSERT);
-                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
-                    load = new SinkOperator();
-                    load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
-                    isTransactionalWrite = false;
-                    break;
-                }
-            }
-            topOp = load;
-        }
+		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
+				isTransactionalWrite, compiledDeclarations);
+		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
+				plan, metadataProvider);
+		return planAndMetadata;
+	}
 
-        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-        AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId, isTransactionalWrite,
-                compiledDeclarations);
-        ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
-        return planAndMetadata;
-    }
+	private FileSplit getDefaultOutputFileLocation() throws MetadataException {
+		if (AsterixProperties.INSTANCE.getOutputDir() == null) {
+			throw new MetadataException(
+					"Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
+		}
+		String filePath = AsterixProperties.INSTANCE.getOutputDir()
+				+ System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
+				+ outputFileID.incrementAndGet();
+		return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(),
+				new FileReference(new File(filePath)));
+	}
 
-    private FileSplit getDefaultOutputFileLocation() throws MetadataException {
-        if (AsterixProperties.INSTANCE.getOutputDir() == null) {
-            throw new MetadataException(
-                    "Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
-        }
-        String filePath = AsterixProperties.INSTANCE.getOutputDir() + System.getProperty("file.separator")
-                + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
-        return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(), new FileReference(new File(filePath)));
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		LogicalVariable v = context.newVar(fc.getVarExpr());
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v = context.newVar(fc.getVarExpr());
+		Expression inExpr = fc.getInExpr();
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+				inExpr, tupSource);
+		ILogicalOperator returnedOp;
 
-        Expression inExpr = fc.getInExpr();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
-        ILogicalOperator returnedOp;
+		if (fc.getPosVarExpr() == null) {
+			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.getInputs().add(eo.second);
 
-        if (fc.getPosVarExpr() == null) {
-            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.getInputs().add(eo.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		LogicalVariable v;
+		ILogicalOperator returnedOp;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v;
-        ILogicalOperator returnedOp;
+		switch (lc.getBindingExpr().getKind()) {
+		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.getInputs().add(tupSource);
+			break;
+		}
+		default: {
+			v = context.newVar(lc.getVarExpr());
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					lc.getBindingExpr(), tupSource);
+			returnedOp = new AssignOperator(v,
+					new MutableObject<ILogicalExpression>(eo.first));
+			returnedOp.getInputs().add(eo.second);
+			break;
+		}
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+	}
 
-        switch (lc.getBindingExpr().getKind()) {
-            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.getInputs().add(tupSource);
-                break;
-            }
-            default: {
-                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
-                        tupSource);
-                v = context.newVar(lc.getVarExpr());
-                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
-                returnedOp.getInputs().add(eo.second);
-                break;
-            }
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
+			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> flworPlan = tupSource;
+		boolean isTop = context.isTopFlwor();
+		if (isTop) {
+			context.setTopFlwor(false);
+		}
+		for (Clause c : flwor.getClauseList()) {
+			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
+					flworPlan);
+			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+		}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> flworPlan = tupSource;
-        boolean isTop = context.isTopFlwor();
-        if (isTop) {
-            context.setTopFlwor(false);
-        }
-        for (Clause c : flwor.getClauseList()) {
-            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
-            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-        }
+		Expression r = flwor.getReturnExpr();
+		boolean noFlworClause = flwor.noForClause();
 
-        Expression r = flwor.getReturnExpr();
-        boolean noFlworClause = flwor.noForClause();
+		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+			VariableExpr v = (VariableExpr) r;
+			LogicalVariable var = context.getVar(v.getVar().getId());
 
-        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-            VariableExpr v = (VariableExpr) r;
-            LogicalVariable var = context.getVar(v.getVar().getId());
+			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+		} else {
+			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
+					flworPlan.getValue());
+			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
+					baseOp);
+			ILogicalOperator rOp = rRes.first;
+			ILogicalOperator resOp;
+			if (expressionNeedsNoNesting(r)) {
+				baseOp.setValue(flworPlan.getValue());
+				resOp = rOp;
+			} else {
+				SubplanOperator s = new SubplanOperator(rOp);
+				s.getInputs().add(flworPlan);
+				resOp = s;
+				baseOp.setValue(new NestedTupleSourceOperator(
+						new MutableObject<ILogicalOperator>(s)));
+			}
+			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
+					resOp);
+			return produceFlwrResult(noFlworClause, isTop, resOpRef,
+					rRes.second);
+		}
+	}
 
-        } else {
-            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
-            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
-            ILogicalOperator rOp = rRes.first;
-            ILogicalOperator resOp;
-            if (expressionNeedsNoNesting(r)) {
-                baseOp.setValue(flworPlan.getValue());
-                resOp = rOp;
-            } else {
-                SubplanOperator s = new SubplanOperator(rOp);
-                s.getInputs().add(flworPlan);
-                resOp = s;
-                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
-            }
-            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
-            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
-        }
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
+			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				fa.getExpr(), tupSource);
+		LogicalVariable v = context.newVar();
+		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+				FunctionUtils
+						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+		fldAccess.getArguments().add(
+				new MutableObject<ILogicalExpression>(p.first));
+		ILogicalExpression faExpr = new ConstantExpression(
+				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+		fldAccess.getArguments().add(
+				new MutableObject<ILogicalExpression>(faExpr));
+		AssignOperator a = new AssignOperator(v,
+				new MutableObject<ILogicalExpression>(fldAccess));
+		a.getInputs().add(p.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
-        LogicalVariable v = context.newVar();
-        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
-                .getValue())));
-        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
-        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
-        a.getInputs().add(p.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+	}
 
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
+			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				ia.getExpr(), tupSource);
+		LogicalVariable v = context.newVar();
+		AbstractFunctionCallExpression f;
+		int i = ia.getIndex();
+		if (i == IndexAccessor.ANY) {
+			f = new ScalarFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+			f.getArguments()
+					.add(new MutableObject<ILogicalExpression>(p.first));
+		} else {
+			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)))));
+		}
+		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> visitIndexAccessor(IndexAccessor ia,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
-        LogicalVariable v = context.newVar();
-        AbstractFunctionCallExpression f;
-        int i = ia.getIndex();
-        if (i == IndexAccessor.ANY) {
-            f = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        } else {
-            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)))));
-        }
-        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)
+			throws AsterixException {
+		LogicalVariable v = context.newVar();
+		AsterixFunction fid = fcall.getIdent();
+		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v = context.newVar();
-        AsterixFunction fid = fcall.getIdent();
-        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-        Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression expr : fcall.getExprList()) {
+			switch (expr.getKind()) {
+			case VARIABLE_EXPRESSION: {
+				LogicalVariable var = context.getVar(((VariableExpr) expr)
+						.getVar().getId());
+				args.add(new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(var)));
+				break;
+			}
+			case LITERAL_EXPRESSION: {
+				LiteralExpr val = (LiteralExpr) expr;
+				args.add(new MutableObject<ILogicalExpression>(
+						new ConstantExpression(
+								new AsterixConstantValue(ConstantHelper
+										.objectFromLiteral(val.getValue())))));
+				break;
+			}
+			default: {
+				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+						expr, topOp);
+				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
+						.getValue();
+				args.add(new MutableObject<ILogicalExpression>(eo.first));
+				if (o1 != null
+						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
+								o1, topOp))) {
+					topOp = eo.second;
+				}
+				break;
+			}
+			}
+		}
 
-        for (Expression expr : fcall.getExprList()) {
-            switch (expr.getKind()) {
-                case VARIABLE_EXPRESSION: {
-                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
-                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
-                    break;
-                }
-                case LITERAL_EXPRESSION: {
-                    LiteralExpr val = (LiteralExpr) expr;
-                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                            ConstantHelper.objectFromLiteral(val.getValue())))));
-                    break;
-                }
-                default: {
-                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
-                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
-                    args.add(new MutableObject<ILogicalExpression>(eo.first));
-                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
-                        topOp = eo.second;
-                    }
-                    break;
-                }
-            }
-        }
+		FunctionIdentifier fi = new FunctionIdentifier(
+				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+		FunctionIdentifier builtinAquafi = afi == null ? null : afi
+				.getFunctionIdentifier();
 
-        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 {
+			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+					fid.getFunctionName());
+			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
+					.getBuiltinFunctionIdentifier(fi);
+			if (builtinAsterixFi != null) {
+				fi = builtinAsterixFi;
+			}
+		}
+		AbstractFunctionCallExpression f;
+		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
+					args);
+		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+					FunctionUtils.getFunctionInfo(fi), args);
+			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
+					.returnsUniqueValues(fi));
+			f = ufce;
+		} else {
+			f = new ScalarFunctionCallExpression(
+					FunctionUtils.getFunctionInfo(fi), args);
+		}
+		AssignOperator op = new AssignOperator(v,
+				new MutableObject<ILogicalExpression>(f));
+		if (topOp != null) {
+			op.getInputs().add(topOp);
+		}
 
-        if (builtinAquafi != null) {
-            fi = builtinAquafi;
-        } else {
-            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName(), false);
-            FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
-            if (builtinAsterixFi != null) {
-                fi = builtinAsterixFi;
-            }
-        }
-        AbstractFunctionCallExpression f;
-        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
-        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(fi), args);
-            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
-            f = ufce;
-        } else {
-            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
-        }
-        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
-        if (topOp != null) {
-            op.getInputs().add(topOp);
-        }
+		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
+			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
+		// TODO Auto-generated method stub
+		throw new NotImplementedException();
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
-            Mutable<ILogicalOperator> tupSource) {
-        // TODO Auto-generated method stub
-        throw new NotImplementedException();
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
+			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		GroupByOperator gOp = new GroupByOperator();
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+			LogicalVariable v;
+			VariableExpr vexpr = ve.getVar();
+			if (vexpr != null) {
+				v = context.newVar(vexpr);
+			} else {
+				v = context.newVar();
+			}
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					ve.getExpr(), topOp);
+			gOp.addGbyExpression(v, eo.first);
+			topOp = eo.second;
+		}
+		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+			LogicalVariable v;
+			VariableExpr vexpr = ve.getVar();
+			if (vexpr != null) {
+				v = context.newVar(vexpr);
+			} else {
+				v = context.newVar();
+			}
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					ve.getExpr(), topOp);
+			gOp.addDecorExpression(v, eo.first);
+			topOp = eo.second;
+		}
+		gOp.getInputs().add(topOp);
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        GroupByOperator gOp = new GroupByOperator();
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-            LogicalVariable v;
-            VariableExpr vexpr = ve.getVar();
-            if (vexpr != null) {
-                v = context.newVar(vexpr);
-            } else {
-                v = context.newVar();
-            }
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
-            gOp.addGbyExpression(v, eo.first);
-            topOp = eo.second;
-        }
-        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-            LogicalVariable v;
-            VariableExpr vexpr = ve.getVar();
-            if (vexpr != null) {
-                v = context.newVar(vexpr);
-            } else {
-                v = context.newVar();
-            }
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
-            gOp.addDecorExpression(v, eo.first);
-            topOp = eo.second;
-        }
-        gOp.getInputs().add(topOp);
+		for (VariableExpr var : gc.getWithVarList()) {
+			LogicalVariable aggVar = context.newVar();
+			LogicalVariable oldVar = context.getVar(var);
+			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			flArgs.add(new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(oldVar)));
+			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
+					.makeAggregateFunctionExpression(
+							AsterixBuiltinFunctions.LISTIFY, flArgs);
+			AggregateOperator agg = new AggregateOperator(
+					mkSingletonArrayList(aggVar),
+					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
+							fListify)));
 
-        for (VariableExpr var : gc.getWithVarList()) {
-            LogicalVariable aggVar = context.newVar();
-            LogicalVariable oldVar = context.getVar(var);
-            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
-            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
-            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                    AsterixBuiltinFunctions.LISTIFY, flArgs);
-            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
-                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+			agg.getInputs().add(
+					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
+			// the one bound by the aggregation op.
+			context.setVar(var, aggVar);
+		}
 
-            agg.getInputs().add(
-                    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
-            // the one bound by the aggregation op.
-            context.setVar(var, aggVar);
-        }
+		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
+				gc.hasHashGroupByHint());
+		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+	}
 
-        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
-        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		// In the most general case, IfThenElse is translated in the following
+		// way.
+		//
+		// We assign the result of the condition to one variable varCond.
+		// We create one subplan which contains the plan for the "then" branch,
+		// on top of which there is a selection whose condition is varCond.
+		// Similarly, we create one subplan for the "else" branch, in which the
+		// selection is not(varCond).
+		// Finally, we concatenate the results. (??)
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        // In the most general case, IfThenElse is translated in the following
-        // way.
-        //
-        // We assign the result of the condition to one variable varCond.
-        // We create one subplan which contains the plan for the "then" branch,
-        // on top of which there is a selection whose condition is varCond.
-        // Similarly, we create one subplan for the "else" branch, in which the
-        // selection is not(varCond).
-        // Finally, we concatenate the results. (??)
+		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
+				.accept(this, tupSource);
+		ILogicalOperator opCond = pCond.first;
+		LogicalVariable varCond = pCond.second;
 
-        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
-        ILogicalOperator opCond = pCond.first;
-        LogicalVariable varCond = pCond.second;
+		SubplanOperator sp = new SubplanOperator();
+		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
+				new NestedTupleSourceOperator(
+						new MutableObject<ILogicalOperator>(sp)));
 
-        SubplanOperator sp = new SubplanOperator();
-        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
-                new MutableObject<ILogicalOperator>(sp)));
+		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
+				.accept(this, nestedSource);
+		SelectOperator sel1 = new SelectOperator(
+				new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(varCond)));
+		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
-        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);
+		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+				new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(varCond)));
+		SelectOperator sel2 = new SelectOperator(
+				new MutableObject<ILogicalExpression>(notVarCond));
+		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
-        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(varCond)));
-        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
-        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+		ILogicalPlan p1 = new ALogicalPlanImpl(
+				new MutableObject<ILogicalOperator>(sel1));
+		sp.getNestedPlans().add(p1);
+		ILogicalPlan p2 = new ALogicalPlanImpl(
+				new MutableObject<ILogicalOperator>(sel2));
+		sp.getNestedPlans().add(p2);
 
-        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
-        sp.getNestedPlans().add(p1);
-        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
-        sp.getNestedPlans().add(p2);
+		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
+				opCond);
+		sp.getInputs().add(opCondRef);
 
-        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
-        sp.getInputs().add(opCondRef);
+		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)));
+		AssignOperator a = new AssignOperator(resV,
+				new MutableObject<ILogicalExpression>(concatNonNull));
+		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-        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)));
-        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
-        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
+			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+		LogicalVariable var = context.newVar();
+		AssignOperator a = new AssignOperator(var,
+				new MutableObject<ILogicalExpression>(new ConstantExpression(
+						new AsterixConstantValue(ConstantHelper
+								.objectFromLiteral(l.getValue())))));
+		if (tupSource != null) {
+			a.getInputs().add(tupSource);
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-        LogicalVariable var = context.newVar();
-        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
-                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
-        if (tupSource != null) {
-            a.getInputs().add(tupSource);
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
+			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		ArrayList<OperatorType> ops = op.getOpList();
+		int nOps = ops.size();
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        ArrayList<OperatorType> ops = op.getOpList();
-        int nOps = ops.size();
+		if (nOps > 0
+				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+			return visitAndOrOperator(op, tupSource);
+		}
 
-        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-            return visitAndOrOperator(op, tupSource);
-        }
+		ArrayList<Expression> exprs = op.getExprList();
 
-        ArrayList<Expression> exprs = op.getExprList();
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-        Mutable<ILogicalOperator> topOp = tupSource;
+		ILogicalExpression currExpr = null;
+		for (int i = 0; i <= nOps; i++) {
 
-        ILogicalExpression currExpr = null;
-        for (int i = 0; i <= nOps; i++) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					exprs.get(i), topOp);
+			topOp = p.second;
+			ILogicalExpression e = p.first;
+			// now look at the operator
+			if (i < nOps) {
+				if (OperatorExpr.opIsComparison(ops.get(i))) {
+					AbstractFunctionCallExpression c = createComparisonExpression(ops
+							.get(i));
 
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
-            topOp = p.second;
-            ILogicalExpression e = p.first;
-            // now look at the operator
-            if (i < nOps) {
-                if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
+					// chain the operators
+					if (i == 0) {
+						c.getArguments().add(
+								new MutableObject<ILogicalExpression>(e));
+						currExpr = c;
+						if (op.isBroadcastOperand(i)) {
+							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+							bcast.setObject(BroadcastSide.LEFT);
+							c.getAnnotations()
+									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+											bcast);
+						}
+					} else {
+						((AbstractFunctionCallExpression) currExpr)
+								.getArguments()
+								.add(new MutableObject<ILogicalExpression>(e));
+						c.getArguments()
+								.add(new MutableObject<ILogicalExpression>(
+										currExpr));
+						currExpr = c;
+						if (i == 1 && op.isBroadcastOperand(i)) {
+							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+							bcast.setObject(BroadcastSide.RIGHT);
+							c.getAnnotations()
+									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+											bcast);
+						}
+					}
+				} else {
+					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
+							.get(i));
 
-                    // chain the operators
-                    if (i == 0) {
-                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
-                        currExpr = c;
-                        if (op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.LEFT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
-                                new MutableObject<ILogicalExpression>(e));
-                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
-                        currExpr = c;
-                        if (i == 1 && op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.RIGHT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    }
-                } else {
-                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
+					if (i == 0) {
+						f.getArguments().add(
+								new MutableObject<ILogicalExpression>(e));
+						currExpr = f;
+					} else {
+						((AbstractFunctionCallExpression) currExpr)
+								.getArguments()
+								.add(new MutableObject<ILogicalExpression>(e));
+						f.getArguments()
+								.add(new MutableObject<ILogicalExpression>(
+										currExpr));
+						currExpr = f;
+					}
+				}
+			} else { // don't forget the last expression...
+				((AbstractFunctionCallExpression) currExpr).getArguments().add(
+						new MutableObject<ILogicalExpression>(e));
+				if (i == 1 && op.isBroadcastOperand(i)) {
+					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+					bcast.setObject(BroadcastSide.RIGHT);
+					((AbstractFunctionCallExpression) currExpr)
+							.getAnnotations()
+							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+									bcast);
+				}
+			}
+		}
 
-                    if (i == 0) {
-                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
-                        currExpr = f;
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
-                                new MutableObject<ILogicalExpression>(e));
-                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
-                        currExpr = f;
-                    }
-                }
-            } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments()
-                        .add(new MutableObject<ILogicalExpression>(e));
-                if (i == 1 && op.isBroadcastOperand(i)) {
-                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                    bcast.setObject(BroadcastSide.RIGHT);
-                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
-                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                }
-            }
-        }
+		LogicalVariable assignedVar = context.newVar();
+		AssignOperator a = new AssignOperator(assignedVar,
+				new MutableObject<ILogicalExpression>(currExpr));
 
-        LogicalVariable assignedVar = context.newVar();
-        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
+		a.getInputs().add(topOp);
 
-        a.getInputs().add(topOp);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
+			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		OrderOperator ord = new OrderOperator();
+		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression e : oc.getOrderbyList()) {
+			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)));
+			topOp = p.second;
+		}
+		ord.getInputs().add(topOp);
+		if (oc.getNumTuples() > 0) {
+			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
+					oc.getNumTuples());
+		}
+		if (oc.getNumFrames() > 0) {
+			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
+					oc.getNumFrames());
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+	}
 
-        OrderOperator ord = new OrderOperator();
-        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (Expression e : oc.getOrderbyList()) {
-            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)));
-            topOp = p.second;
-        }
-        ord.getInputs().add(topOp);
-        if (oc.getNumTuples() > 0) {
-            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
-        }
-        if (oc.getNumFrames() > 0) {
-            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
+			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> topOp = tupSource;
+		ILogicalOperator firstOp = null;
+		Mutable<ILogicalOperator> lastOp = null;
 
-        ILogicalOperator firstOp = null;
-        Mutable<ILogicalOperator> lastOp = null;
+		for (QuantifiedPair qt : qe.getQuantifiedList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
+					qt.getExpr(), topOp);
+			topOp = eo1.second;
+			LogicalVariable uVar = context.newVar(qt.getVarExpr());
+			ILogicalOperator u = new UnnestOperator(uVar,
+					new MutableObject<ILogicalExpression>(
+							makeUnnestExpression(eo1.first)));
 
-        for (QuantifiedPair qt : qe.getQuantifiedList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
-            topOp = eo1.second;
-            LogicalVariable uVar = context.newVar(qt.getVarExpr());
-            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
-                    makeUnnestExpression(eo1.first)));
+			if (firstOp == null) {
+				firstOp = u;
+			}
+			if (lastOp != null) {
+				u.getInputs().add(lastOp);
+			}
+			lastOp = new MutableObject<ILogicalOperator>(u);
+		}
 
-            if (firstOp == null) {
-                firstOp = u;
-            }
-            if (lastOp != null) {
-                u.getInputs().add(lastOp);
-            }
-            lastOp = new MutableObject<ILogicalOperator>(u);
-        }
+		// We make all the unnest correspond. to quantif. vars. sit on top
+		// in the hope of enabling joins & other optimiz.
+		firstOp.getInputs().add(topOp);
+		topOp = lastOp;
 
-        // We make all the unnest correspond. to quantif. vars. sit on top
-        // in the hope of enabling joins & other optimiz.
-        firstOp.getInputs().add(topOp);
-        topOp = lastOp;
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
+				qe.getSatisfiesExpr(), topOp);
 
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
-
-        AggregateFunctionCallExpression fAgg;
-        SelectOperator s;
-        if (qe.getQuantifier() == Quantifier.SOME) {
-            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
-            s.getInputs().add(eo2.second);
-            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-                    new ArrayList<Mutable<ILogicalExpression>>());
-        } else { // EVERY
-            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
-            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
-            s.getInputs().add(eo2.second);
-            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
-                    new ArrayList<Mutable<ILogicalExpression>>());
-        }
-        LogicalVariable qeVar = context.newVar();
-        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
-        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-    }
+		AggregateFunctionCallExpression fAgg;
+		SelectOperator s;
+		if (qe.getQuantifier() == Quantifier.SOME) {
+			s = new SelectOperator(new MutableObject<ILogicalExpression>(
+					eo2.first));
+			s.getInputs().add(eo2.second);
+			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+					new ArrayList<Mutable<ILogicalExpression>>());
+		} else { // EVERY
+			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+			s = new SelectOperator(new MutableObject<ILogicalExpression>(
+					new ScalarFunctionCallExpression(FunctionUtils
+							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+							satExprList)));
+			s.getInputs().add(eo2.second);
+			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+					AsterixBuiltinFunctions.EMPTY_STREAM,
+					new ArrayList<Mutable<ILogicalExpression>>());
+		}
+		LogicalVariable qeVar = context.newVar();
+		AggregateOperator a = new AggregateOperator(
+				mkSingletonArrayList(qeVar),
+				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
+						fAgg)));
+		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        return q.getBody().accept(this, tupSource);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		return q.getBody().accept(this, tupSource);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (FieldBinding fb : rc.getFbList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
-            topOp = eo1.second;
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
-            topOp = eo2.second;
-        }
-        a.getInputs().add(topOp);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
+			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+				FunctionUtils
+						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a = new AssignOperator(v1,
+				new MutableObject<ILogicalExpression>(f));
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (FieldBinding fb : rc.getFbList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
+					fb.getLeftExpr(), topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo1.first));
+			topOp = eo1.second;
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
+					fb.getRightExpr(), topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo2.first));
+			topOp = eo2.second;
+		}
+		a.getInputs().add(topOp);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (Expression expr : lc.getExprList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
-            topOp = eo.second;
-        }
-        a.getInputs().add(topOp);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
+			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(fid));
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a = new AssignOperator(v1,
+				new MutableObject<ILogicalExpression>(f));
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression expr : lc.getExprList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					expr, topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo.first));
+			topOp = eo.second;
+		}
+		a.getInputs().add(topOp);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Expression expr = u.getExpr();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a;
-        if (u.getSign() == Sign.POSITIVE) {
-            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
-        } else {
-            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
-            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-        }
-        a.getInputs().add(eo.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		Expression expr = u.getExpr();
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+				expr, tupSource);
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a;
+		if (u.getSign() == Sign.POSITIVE) {
+			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
+					eo.first));
+		} else {
+			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+			m.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo.first));
+			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+		}
+		a.getInputs().add(eo.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-        // 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)));
-        a.getInputs().add(tupSource);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
+			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+		// 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)));
+		a.getInputs().add(tupSource);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
-        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
-        s.getInputs().add(p.second);
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
+			WhereClause w, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				w.getWhereExpr(), tupSource);
+		SelectOperator s = new SelectOperator(
+				new MutableObject<ILogicalExpression>(p.first));
+		s.getInputs().add(p.second);
 
-        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-    }
+		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
-        LimitOperator opLim;
-        Expression offset = lc.getOffset();
-        if (offset != null) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
-            opLim = new LimitOperator(p1.first, p2.first);
-            opLim.getInputs().add(p2.second);
-        } else {
-            opLim = new LimitOperator(p1.first);
-            opLim.getInputs().add(p1.second);
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
+			LimitClause lc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
+				lc.getLimitExpr(), tupSource);
+		LimitOperator opLim;
+		Expression offset = lc.getOffset();
+		if (offset != null) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
+					offset, p1.second);
+			opLim = new LimitOperator(p1.first, p2.first);
+			opLim.getInputs().add(p2.second);
+		} else {
+			opLim = new LimitOperator(p1.first);
+			opLim.getInputs().add(p1.second);
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
-        DieOperator opDie = new DieOperator(p1.first);
-        opDie.getInputs().add(p1.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
+				lc.getDieExpr(), tupSource);
+		DieOperator opDie = new DieOperator(p1.first);
+		opDie.getInputs().add(p1.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-        Mutable<ILogicalOperator> input = null;
-        for (Expression expr : dc.getDistinctByExpr()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
-            exprList.add(new MutableObject<ILogicalExpression>(p.first));
-            input = p.second;
-        }
-        DistinctOperator opDistinct = new DistinctOperator(exprList);
-        opDistinct.getInputs().add(input);
-        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
+			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+		Mutable<ILogicalOperator> input = null;
+		for (Expression expr : dc.getDistinctByExpr()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					expr, tupSource);
+			exprList.add(new MutableObject<ILogicalExpression>(p.first));
+			input = p.second;
+		}
+		DistinctOperator opDistinct = new DistinctOperator(exprList);
+		opDistinct.getInputs().add(input);
+		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> ts = tupSource;
-        ILogicalOperator lastOp = null;
-        LogicalVariable lastVar = null;
-        boolean first = true;
-        for (Expression e : unionExpr.getExprs()) {
-            if (first) {
-                first = false;
-            } else {
-                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
-            }
-            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-            if (lastOp == null) {
-                lastOp = p1.first;
-                lastVar = p1.second;
-            } else {
-                LogicalVariable unnestVar1 = context.newVar();
-                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
-                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
-                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-                LogicalVariable unnestVar2 = context.newVar();
-                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
-                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
-                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
-                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-                        1);
-                LogicalVariable resultVar = context.newVar();
-                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-                        unnestVar1, unnestVar2, resultVar);
-                varMap.add(triple);
-                UnionAllOperator unionOp = new UnionAllOperator(varMap);
-                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
-                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
-                lastVar = resultVar;
-                lastOp = unionOp;
-            }
-        }
-        LogicalVariable aggVar = context.newVar();
-        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
-        aggregVars.add(aggVar);
-        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
-        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                AsterixBuiltinFunctions.LISTIFY, afcExprs);
-        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
+			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> ts = tupSource;
+		ILogicalOperator lastOp = null;
+		LogicalVariable lastVar = null;
+		boolean first = true;
+		for (Expression e : unionExpr.getExprs()) {
+			if (first) {
+				first = false;
+			} else {
+				ts = new MutableObject<ILogicalOperator>(
+						new EmptyTupleSourceOperator());
+			}
+			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+			if (lastOp == null) {
+				lastOp = p1.first;
+				lastVar = p1.second;
+			} else {
+				LogicalVariable unnestVar1 = context.newVar();
+				UnnestOperator unnest1 = new UnnestOperator(
+						unnestVar1,
+						new MutableObject<ILogicalExpression>(
+								makeUnnestExpression(new VariableReferenceExpression(
+										lastVar))));
+				unnest1.getInputs().add(
+						new MutableObject<ILogicalOperator>(lastOp));
+				LogicalVariable unnestVar2 = context.newVar();
+				UnnestOperator unnest2 = new UnnestOperator(
+						unnestVar2,
+						new MutableObject<ILogicalExpression>(
+								makeUnnestExpression(new VariableReferenceExpression(
+										p1.second))));
+				unnest2.getInputs().add(
+						new MutableObject<ILogicalOperator>(p1.first));
+				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+						1);
+				LogicalVariable resultVar = context.newVar();
+				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+						unnestVar1, unnestVar2, resultVar);
+				varMap.add(triple);
+				UnionAllOperator unionOp = new UnionAllOperator(varMap);
+				unionOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(unnest1));
+				unionOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(unnest2));
+				lastVar = resultVar;
+				lastOp = unionOp;
+			}
+		}
+		LogicalVariable aggVar = context.newVar();
+		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
+				1);
+		aggregVars.add(aggVar);
+		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
+				1);
+		afcExprs.add(new MutableObject<ILogicalExpression>(
+				new VariableReferenceExpression(lastVar)));
+		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
+				.makeAggregateFunctionExpression(
+						AsterixBuiltinFunctions.LISTIFY, afcExprs);
+		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
+				1);
+		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+	}
 
-    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
-        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-        return new ScalarFunctionCallExpression(finfo);
-    }
+	private AbstractFunctionCallExpression createComparisonExpression(
+			OperatorType t) {
+		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+		return new ScalarFunctionCallExpression(finfo);
+	}
 
-    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-        switch (t) {
-            case EQ: {
-                return AlgebricksBuiltinFunctions.EQ;
-            }
-            case NEQ: {
-                return AlgebricksBuiltinFunctions.NEQ;
-            }
-            case GT: {
-                return AlgebricksBuiltinFunctions.GT;
-            }
-            case GE: {
-                return AlgebricksBuiltinFunctions.GE;
-            }
-            case LT: {
-                return AlgebricksBuiltinFunctions.LT;
-            }
-            case LE: {
-                return AlgebricksBuiltinFunctions.LE;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-    }
+	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+		switch (t) {
+		case EQ: {
+			return AlgebricksBuiltinFunctions.EQ;
+		}
+		case NEQ: {
+			return AlgebricksBuiltinFunctions.NEQ;
+		}
+		case GT: {
+			return AlgebricksBuiltinFunctions.GT;
+		}
+		case GE: {
+			return AlgebricksBuiltinFunctions.GE;
+		}
+		case LT: {
+			return AlgebricksBuiltinFunctions.LT;
+		}
+		case LE: {
+			return AlgebricksBuiltinFunctions.LE;
+		}
+		default: {
+			throw new IllegalStateException();
+		}
+		}
+	}
 
-    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
-            throws AsterixException {
+	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
+			OperatorType t) throws AsterixException {
 
-        FunctionIdentifier fid = null;
-        switch (t) {
-            case PLUS: {
-                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-                break;
-            }
-            case MINUS: {
-                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-                break;
-            }
-            case MUL: {
-                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-                break;
-            }
-            case DIV: {
-                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-                break;
-            }
-            case MOD: {
-                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-                break;
-            }
-            case IDIV: {
-                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-                break;
-            }
-            case CARET: {
-                fid = AsterixBuiltinFunctions.CARET;
-                break;
-            }
-            case AND: {
-                fid = AlgebricksBuiltinFunctions.AND;
-                break;
-            }
-            case OR: {
-                fid = AlgebricksBuiltinFunctions.OR;
-                break;
-            }
-            case FUZZY_EQ: {
-                fid = AsterixBuiltinFunctions.FUZZY_EQ;
-                break;
-            }
+		FunctionIdentifier fid = null;
+		switch (t) {
+		case PLUS: {
+			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+			break;
+		}
+		case MINUS: {
+			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+			break;
+		}
+		case MUL: {
+			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+			break;
+		}
+		case DIV: {
+			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+			break;
+		}
+		case MOD: {
+			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+			break;
+		}
+		case IDIV: {
+			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+			break;
+		}
+		case CARET: {
+			fid = AsterixBuiltinFunctions.CARET;
+			break;
+		}
+		case AND: {
+			fid = AlgebricksBuiltinFunctions.AND;
+			break;
+		}
+		case OR: {
+			fid = AlgebricksBuiltinFunctions.OR;
+			break;
+		}
+		case FUZZY_EQ: {
+			fid = AsterixBuiltinFunctions.FUZZY_EQ;
+			break;
+		}
 
-            default: {
-                throw new NotImplementedException("Operator " + t + " is not yet implemented");
-            }
-        }
-        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
-    }
+		default: {
+			throw new NotImplementedException("Operator " + t
+					+ " is not yet implemented");
+		}
+		}
+		return new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(fid));
+	}
 
-    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
-        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-        if (inp == null || inp.size() != 1) {
-            return false;
-        }
-        return inp.get(0) == childCandidate;
-    }
+	private static boolean hasOnlyChild(ILogicalOperator parent,
+			Mutable<ILogicalOperator> childCandidate) {
+		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+		if (inp == null || inp.size() != 1) {
+			return false;
+		}
+		return inp.get(0) == childCandidate;
+	}
 
-    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
-            Mutable<ILogicalOperator> topOp) throws AsterixException {
-        switch (expr.getKind()) {
-            case VARIABLE_EXPRESSION: {
-                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
-                        .getVar().getId()));
-                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
-            }
-            case LITERAL_EXPRESSION: {
-                LiteralExpr val = (LiteralExpr) expr;
-                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
-                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
-            }
-            default: {
-                // Mutable<ILogicalOperator> src = new
-                // Mutable<ILogicalOperator>();
-                // Mutable<ILogicalOperator> src = topOp;
-                if (expressionNeedsNoNesting(expr)) {
-                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
-                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
-                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
-                } else {
-                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
+			Expression expr, Mutable<ILogicalOperator> topOp)
+			throws AsterixException {
+		switch (expr.getKind()) {
+		case VARIABLE_EXPRESSION: {
+			VariableReferenceExpression ve = new VariableReferenceExpression(
+					context.getVar(((VariableExpr) expr).getVar().getId()));
+			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
+					topOp);
+		}
+		case LITERAL_EXPRESSION: {
+			LiteralExpr val = (LiteralExpr) expr;
+			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+					new ConstantExpression(new AsterixConstantValue(
+							ConstantHelper.objectFromLiteral(val.getValue()))),
+					topOp);
+		}
+		default: {
+			// Mutable<ILogicalOperator> src = new
+			// Mutable<ILogicalOperator>();
+			// Mutable<ILogicalOperator> src = topOp;
+			if (expressionNeedsNoNesting(expr)) {
+				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+						topOp);
+				ILogicalExpression exp = ((AssignOperator) p.first)
+						.getExpressions().get(0).getValue();
+				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+						exp, p.first.getInputs().get(0));
+			} else {
+				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
+				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+						src);
 
-                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-                        // src.setOperator(topOp.getOperator());
-                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
-                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
-                                p.second), top2);
-                    } else {
-                        SubplanOperator s = new SubplanOperator();
-                        s.getInputs().add(topOp);
-                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
-                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
-                        s.setRootOp(planRoot);
-                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
-                                p.second), new MutableObject<ILogicalOperator>(s));
-                    }
-                }
-            }
-        }
+				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+					// src.setOperator(topOp.getOperator());
+					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
+							p.first);
+					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+							new VariableReferenceExpression(p.second), top2);
+				} else {
+					SubplanOperator s = new SubplanOperator();
+					s.getInputs().add(topOp);
+					src.setValue(new NestedTupleSourceOperator(
+							new MutableObject<ILogicalOperator>(s)));
+					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
+							p.first);
+					s.setRootOp(planRoot);
+					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+							new VariableReferenceExpression(p.second),
+							new MutableObject<ILogicalOperator>(s));
+				}
+			}
+		}
+		}
 
-    }
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
-            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-        if (isTop) {
-            ProjectOperator pr = new ProjectOperator(resVar);
-            pr.getInputs().add(resOpRef);
-            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
+			boolean noForClause, boolean isTop,
+			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+		if (isTop) {
+			ProjectOperator pr = new ProjectOperator(resVar);
+			pr.getInputs().add(resOpRef);
+			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-        } else if (noForClause) {
-            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
-        } else {
-            return aggListify(resVar, resOpRef, false);
-        }
-    }
+		} else if (noForClause) {
+			return new Pair<ILogicalOperator, LogicalVariable>(
+					resOpRef.getValue(), resVar);
+		} else {
+			return aggListify(resVar, resOpRef, false);
+		}
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
-            boolean bProject) {
-        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
-        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)));
-        agg.getInputs().add(opRef);
-        ILogicalOperator res;
-        if (bProject) {
-            ProjectOperator pr = new ProjectOperator(varListified);
-            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-            res = pr;
-        } else {
-            res = agg;
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-    }
+	private Pair<ILogicalOperator, LogicalVariable> aggListify(
+			LogicalVariable var, Mutable<ILogicalOperator> opRef,
+			boolean bProject) {
+		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
+				.makeAggregateFunctionExpression(
+						AsterixBuiltinFunctions.LISTIFY,
+						new ArrayList<Mutable<ILogicalExpression>>());
+		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)));
+		agg.getInputs().add(opRef);
+		ILogicalOperator res;
+		if (bProject) {
+			ProjectOperator pr = new ProjectOperator(varListified);
+			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+			res = pr;
+		} else {
+			res = agg;
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        ArrayList<OperatorType> ops = op.getOpList();
-        int nOps = ops.size();
+	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
+			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		ArrayList<OperatorType> ops = op.getOpList();
+		int nOps = ops.size();
 
-        ArrayList<Expression> exprs = op.getExprList();
+		ArrayList<Expression> exprs = op.getExprList();
 
-        Mutable<ILogicalOperator> topOp = tupSource;
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-        OperatorType opLogical = ops.get(0);
-        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+		OperatorType opLogical = ops.get(0);
+		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-        for (int i = 0; i <= nOps; i++) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
-            topOp = p.second;
-            // now look at the operator
-            if (i < nOps) {
-                if (ops.get(i) != opLogical) {
-                    throw new TranslationException("Unexpected operator " + ops.get(i)
-                            + " in an OperatorExpr starting with " + opLogical);
-                }
-            }
-            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        }
+		for (int i = 0; i <= nOps; i++) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					exprs.get(i), topOp);
+			topOp = p.second;
+			// now look at the operator
+			if (i < nOps) {
+				if (ops.get(i) != opLogical) {
+					throw new TranslationException("Unexpected operator "
+							+ ops.get(i) + " in an OperatorExpr starting with "
+							+ opLogical);
+				}
+			}
+			f.getArguments()
+					.add(new MutableObject<ILogicalExpression>(p.first));
+		}
 
-        LogicalVariable assignedVar = context.newVar();
-        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
-        a.getInputs().add(topOp);
+		LogicalVariable assignedVar = context.newVar();
+		AssignOperator a = new AssignOperator(assignedVar,
+				new MutableObject<ILogicalExpression>(f));
+		a.getInputs().add(topOp);
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-    }
+	}
 
-    private static boolean expressionNeedsNoNesting(Expression expr) {
-        Kind k = expr.getKind();
-        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
-                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
-                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
-    }
+	private static boolean expressionNeedsNoNesting(Expression expr) {
+		Kind k = expr.getKind();
+		return k == Kind.LITERAL_EXPRESSION
+				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
+				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
+				|| k == Kind.OP_EXPRESSION
+				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
+				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
+				|| k == Kind.UNARY_EXPRESSION;
+	}
 
-    private <T> ArrayList<T> mkSingletonArrayList(T item) {
-        ArrayList<T> array = new ArrayList<T>(1);
-        array.add(item);
-        return array;
-    }
+	private <T> ArrayList<T> mkSingletonArrayList(T item) {
+		ArrayList<T> array = new ArrayList<T>(1);
+		array.add(item);
+		return array;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
+			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
+			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
+			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
+			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+			WriteFromQueryResultStatement stmtLoad,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
+			DropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
+			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
+			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
+			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
+			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-        switch (expr.getExpressionTag()) {
-            case VARIABLE: {
-                return new UnnestingFunctionCallExpression(
-                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-                        new MutableObject<ILogicalExpression>(expr));
-            }
-            case FUNCTION_CALL: {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                if (fce.getKind() == FunctionKind.UNNEST) {
-                    return expr;
-                } else {
-                    return new UnnestingFunctionCallExpression(
-                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-                            new MutableObject<ILogicalExpression>(expr));
-                }
-            }
-            default: {
-                return expr;
-            }
-        }
-    }
+	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+		switch (expr.getExpressionTag()) {
+		case VARIABLE: {
+			return new UnnestingFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+					new MutableObject<ILogicalExpression>(expr));
+		}
+		case FUNCTION_CALL: {
+			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+			if (fce.getKind() == FunctionKind.UNNEST) {
+				return expr;
+			} else {
+				return new UnnestingFunctionCallExpression(
+						FunctionUtils
+								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+						new MutableObject<ILogicalExpression>(expr));
+			}
+		}
+		default: {
+			return expr;
+		}
+		}
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
+			InsertStatement insert, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
+			DeleteStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
+			UpdateStatement update, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
+			UpdateClause del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
+			DataverseDecl dv, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
+			DatasetDecl dd, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
+			SetStatement ss, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
+			WriteStatement ws, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+			WriteFromQueryResultStatement stmtLoad,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
+			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
+			IndexDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
+			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
+			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
+			TypeDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visit(
+			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
+			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
+			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 }
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 e6aaaad..4fc1fc8 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
@@ -81,12 +81,12 @@
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
 import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
@@ -94,6 +94,7 @@
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.AsterixFunction;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -144,1328 +145,1584 @@
 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.
- * 
+ * 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
-        IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator
+		implements
+		IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-    private static final Logger LOGGER = Logger.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
+	private static final Logger LOGGER = Logger
+			.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
 
-    private class MetaScopeLogicalVariable {
-        private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
+	private class MetaScopeLogicalVariable {
+		private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
 
-        public VariableReferenceExpression getVariableReferenceExpression(Identifier id) throws AsterixException {
-            LogicalVariable var = map.get(id);
-            LOGGER.fine("get:" + id + ":" + var);
-            if (var == null) {
-                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
-            }
-            return new VariableReferenceExpression(var);
-        }
+		public VariableReferenceExpression getVariableReferenceExpression(
+				Identifier id) throws AsterixException {
+			LogicalVariable var = map.get(id);
+			LOGGER.fine("get:" + id + ":" + var);
+			if (var == null) {
+				throw new AsterixException("Identifier " + id
+						+ " not found in AQL+ meta-scope.");
+			}
+			return new VariableReferenceExpression(var);
+		}
 
-        public void put(Identifier id, LogicalVariable var) {
-            LOGGER.fine("put:" + id + ":" + var);
-            map.put(id, var);
-        }
-    }
+		public void put(Identifier id, LogicalVariable var) {
+			LOGGER.fine("put:" + id + ":" + var);
+			map.put(id, var);
+		}
+	}
 
-    private class MetaScopeILogicalOperator {
-        private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
+	private class MetaScopeILogicalOperator {
+		private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
 
-        public ILogicalOperator get(Identifier id) throws AsterixException {
-            ILogicalOperator op = map.get(id);
-            if (op == null) {
-                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
-            }
-            return op;
-        }
+		public ILogicalOperator get(Identifier id) throws AsterixException {
+			ILogicalOperator op = map.get(id);
+			if (op == null) {
+				throw new AsterixException("Identifier " + id
+						+ " not found in AQL+ meta-scope.");
+			}
+			return op;
+		}
 
-        public void put(Identifier id, ILogicalOperator op) {
-            LOGGER.fine("put:" + id + ":" + op);
-            map.put(id, op);
-        }
-    }
+		public void put(Identifier id, ILogicalOperator op) {
+			LOGGER.fine("put:" + id + ":" + op);
+			map.put(id, op);
+		}
+	}
 
-    private final long txnId;
-    private final MetadataTransactionContext mdTxnCtx;
-    private TranslationContext context;
-    private String outputDatasetName;
-    private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
-    private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
-    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+	private final long txnId;
+	private final MetadataTransactionContext mdTxnCtx;
+	private TranslationContext context;
+	private String outputDatasetName;
+	private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
+	private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
+	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-    public AqlPlusExpressionToPlanTranslator(long txnId, MetadataTransactionContext mdTxnCtx,
-            Counter currentVarCounter, String outputDatasetName) {
-        this.txnId = txnId;
-        this.mdTxnCtx = mdTxnCtx;
-        this.context = new TranslationContext(currentVarCounter);
-        this.outputDatasetName = outputDatasetName;
-        this.context.setTopFlwor(false);
-    }
+	public AqlPlusExpressionToPlanTranslator(long txnId,
+			MetadataTransactionContext mdTxnCtx, Counter currentVarCounter,
+			String outputDatasetName) {
+		this.txnId = txnId;
+		this.mdTxnCtx = mdTxnCtx;
+		this.context = new TranslationContext(currentVarCounter);
+		this.outputDatasetName = outputDatasetName;
+		this.context.setTopFlwor(false);
+	}
 
-    public int getVarCounter() {
-        return context.getVarCounter();
-    }
+	public int getVarCounter() {
+		return context.getVarCounter();
+	}
 
-    public ILogicalPlanAndMetadata translate(Query expr) throws AlgebricksException, AsterixException {
-        return translate(expr, null);
-    }
+	public ILogicalPlanAndMetadata translate(Query expr)
+			throws AlgebricksException, AsterixException {
+		return translate(expr, null);
+	}
 
-    public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
-            throws AlgebricksException, AsterixException {
-        if (expr == null) {
-            return null;
-        }
-        if (compiledDeclarations == null) {
-            compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
-        }
-        if (!compiledDeclarations.isConnectedToDataverse())
-            compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
-        IDataFormat format = compiledDeclarations.getFormat();
-        if (format == null) {
-            throw new AlgebricksException("Data format has not been set.");
-        }
-        format.registerRuntimeFunctions();
-        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
-                new EmptyTupleSourceOperator()));
+	public ILogicalPlanAndMetadata translate(Query expr,
+			AqlCompiledMetadataDeclarations compiledDeclarations)
+			throws AlgebricksException, AsterixException {
+		if (expr == null) {
+			return null;
+		}
+		if (compiledDeclarations == null) {
+			compiledDeclarations = compileMetadata(mdTxnCtx,
+					expr.getPrologDeclList(), true);
+		}
+		if (!compiledDeclarations.isConnectedToDataverse())
+			compiledDeclarations.connectToDataverse(compiledDeclarations
+					.getDataverseName());
+		IDataFormat format = compiledDeclarations.getFormat();
+		if (format == null) {
+			throw new AlgebricksException("Data format has not been set.");
+		}
+		format.registerRuntimeFunctions();
+		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+				new MutableObject<ILogicalOperator>(
+						new EmptyTupleSourceOperator()));
 
-        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-        boolean isTransactionalWrite = false;
-        ILogicalOperator topOp = p.first;
-        ProjectOperator project = (ProjectOperator) topOp;
-        LogicalVariable resVar = project.getVariables().get(0);
-        if (outputDatasetName == null) {
-            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
-            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
-            FileSplitSinkId fssi = new FileSplitSinkId(compiledDeclarations.getOutputFile());
-            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-            topOp = new WriteOperator(writeExprList, sink);
-            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-        } else {
-            AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
-            if (adecl == null) {
-                throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
-            }
+		boolean isTransactionalWrite = false;
+		ILogicalOperator topOp = p.first;
+		ProjectOperator project = (ProjectOperator) topOp;
+		LogicalVariable resVar = project.getVariables().get(0);
+		if (outputDatasetName == null) {
+			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			writeExprList.add(new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(resVar)));
+			FileSplitSinkId fssi = new FileSplitSinkId(
+					compiledDeclarations.getOutputFile());
+			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+			topOp = new WriteOperator(writeExprList, sink);
+			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+		} else {
+			Dataset dataset = compiledDeclarations
+					.findDataset(outputDatasetName);
+			if (dataset == null) {
+				throw new AlgebricksException("Cannot find dataset "
+						+ outputDatasetName);
+			}
+			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);
+				AbstractFunctionCallExpression f = partitioner.second
+						.cloneExpression();
+				f.substituteVar(METADATA_DUMMY_VAR, resVar);
+				exprs.add(new MutableObject<ILogicalExpression>(f));
+				LogicalVariable v = context.newVar();
+				vars.add(v);
+				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(v)));
+			}
+			AssignOperator assign = new AssignOperator(vars, exprs);
+			assign.getInputs()
+					.add(new MutableObject<ILogicalOperator>(project));
+		}
 
-            if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
-                throw new AlgebricksException("Cannot write output to an external dataset.");
-            }
-            ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-            ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
-            List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
-            for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner : DatasetUtils
-                    .getPartitioningFunctions(adecl)) {
-                AbstractFunctionCallExpression f = partitioner.second.cloneExpression();
-                f.substituteVar(METADATA_DUMMY_VAR, resVar);
-                exprs.add(new MutableObject<ILogicalExpression>(f));
-                LogicalVariable v = context.newVar();
-                vars.add(v);
-                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
-            }
-            AssignOperator assign = new AssignOperator(vars, exprs);
-            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
-        }
+		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
+				isTransactionalWrite, compiledDeclarations);
+		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
+				plan, metadataProvider);
+		return planAndMetadata;
+	}
 
-        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-        AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId, isTransactionalWrite,
-                compiledDeclarations);
-        ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
-        return planAndMetadata;
-    }
+	public ILogicalPlan translate(List<Clause> clauses)
+			throws AlgebricksException, AsterixException {
 
-    public ILogicalPlan translate(List<Clause> clauses) throws AlgebricksException, AsterixException {
+		if (clauses == null) {
+			return null;
+		}
 
-        if (clauses == null) {
-            return null;
-        }
+		Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(
+				new EmptyTupleSourceOperator());
+		Pair<ILogicalOperator, LogicalVariable> p = null;
+		for (Clause c : clauses) {
+			p = c.accept(this, opRef);
+			opRef = new MutableObject<ILogicalOperator>(p.first);
+		}
 
-        Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
-        Pair<ILogicalOperator, LogicalVariable> p = null;
-        for (Clause c : clauses) {
-            p = c.accept(this, opRef);
-            opRef = new MutableObject<ILogicalOperator>(p.first);
-        }
+		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+		ILogicalOperator topOp = p.first;
 
-        ILogicalOperator topOp = p.first;
+		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+		return plan;
+	}
 
-        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-        return plan;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		LogicalVariable v = context.newVar(fc.getVarExpr());
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v = context.newVar(fc.getVarExpr());
+		Expression inExpr = fc.getInExpr();
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+				inExpr, tupSource);
+		ILogicalOperator returnedOp;
 
-        Expression inExpr = fc.getInExpr();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
-        ILogicalOperator returnedOp;
+		if (fc.getPosVarExpr() == null) {
+			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.getInputs().add(eo.second);
 
-        if (fc.getPosVarExpr() == null) {
-            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.getInputs().add(eo.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		LogicalVariable v;
+		ILogicalOperator returnedOp;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v;
-        ILogicalOperator returnedOp;
+		switch (lc.getBindingExpr().getKind()) {
+		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.getInputs().add(tupSource);
+			break;
+		}
+		default: {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					lc.getBindingExpr(), tupSource);
+			v = context.newVar(lc.getVarExpr());
+			returnedOp = new AssignOperator(v,
+					new MutableObject<ILogicalExpression>(eo.first));
+			returnedOp.getInputs().add(eo.second);
+			break;
+		}
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+	}
 
-        switch (lc.getBindingExpr().getKind()) {
-            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.getInputs().add(tupSource);
-                break;
-            }
-            default: {
-                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
-                        tupSource);
-                v = context.newVar(lc.getVarExpr());
-                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
-                returnedOp.getInputs().add(eo.second);
-                break;
-            }
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
+			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> flworPlan = tupSource;
+		boolean isTop = context.isTopFlwor();
+		if (isTop) {
+			context.setTopFlwor(false);
+		}
+		for (Clause c : flwor.getClauseList()) {
+			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
+					flworPlan);
+			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+		}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> flworPlan = tupSource;
-        boolean isTop = context.isTopFlwor();
-        if (isTop) {
-            context.setTopFlwor(false);
-        }
-        for (Clause c : flwor.getClauseList()) {
-            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
-            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-        }
+		Expression r = flwor.getReturnExpr();
+		boolean noFlworClause = flwor.noForClause();
 
-        Expression r = flwor.getReturnExpr();
-        boolean noFlworClause = flwor.noForClause();
+		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+			VariableExpr v = (VariableExpr) r;
+			LogicalVariable var = context.getVar(v.getVar().getId());
 
-        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-            VariableExpr v = (VariableExpr) r;
-            LogicalVariable var = context.getVar(v.getVar().getId());
+			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+		} else {
+			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
+					flworPlan.getValue());
+			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
+					baseOp);
+			ILogicalOperator rOp = rRes.first;
+			ILogicalOperator resOp;
+			if (expressionNeedsNoNesting(r)) {
+				baseOp.setValue(flworPlan.getValue());
+				resOp = rOp;
+			} else {
+				SubplanOperator s = new SubplanOperator(rOp);
+				s.getInputs().add(flworPlan);
+				resOp = s;
+				baseOp.setValue(new NestedTupleSourceOperator(
+						new MutableObject<ILogicalOperator>(s)));
+			}
+			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
+					resOp);
+			return produceFlwrResult(noFlworClause, isTop, resOpRef,
+					rRes.second);
+		}
+	}
 
-        } else {
-            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
-            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
-            ILogicalOperator rOp = rRes.first;
-            ILogicalOperator resOp;
-            if (expressionNeedsNoNesting(r)) {
-                baseOp.setValue(flworPlan.getValue());
-                resOp = rOp;
-            } else {
-                SubplanOperator s = new SubplanOperator(rOp);
-                s.getInputs().add(flworPlan);
-                resOp = s;
-                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
-            }
-            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
-            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
-        }
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
+			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				fa.getExpr(), tupSource);
+		LogicalVariable v = context.newVar();
+		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+				FunctionUtils
+						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+		fldAccess.getArguments().add(
+				new MutableObject<ILogicalExpression>(p.first));
+		ILogicalExpression faExpr = new ConstantExpression(
+				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+		fldAccess.getArguments().add(
+				new MutableObject<ILogicalExpression>(faExpr));
+		AssignOperator a = new AssignOperator(v,
+				new MutableObject<ILogicalExpression>(fldAccess));
+		a.getInputs().add(p.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
-        LogicalVariable v = context.newVar();
-        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
-                .getValue())));
-        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
-        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
-        a.getInputs().add(p.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+	}
 
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
+			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				ia.getExpr(), tupSource);
+		LogicalVariable v = context.newVar();
+		AbstractFunctionCallExpression f;
+		int i = ia.getIndex();
+		if (i == IndexAccessor.ANY) {
+			f = new ScalarFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+			f.getArguments()
+					.add(new MutableObject<ILogicalExpression>(p.first));
+		} else {
+			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)))));
+		}
+		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> visitIndexAccessor(IndexAccessor ia,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
-        LogicalVariable v = context.newVar();
-        AbstractFunctionCallExpression f;
-        int i = ia.getIndex();
-        if (i == IndexAccessor.ANY) {
-            f = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        } else {
-            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)))));
-        }
-        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)
+			throws AsterixException {
+		LogicalVariable v = context.newVar();
+		AsterixFunction fid = fcall.getIdent();
+		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall,  Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        LogicalVariable v = context.newVar();
-        AsterixFunction fid = fcall.getIdent();
-        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-        Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression expr : fcall.getExprList()) {
+			switch (expr.getKind()) {
+			case VARIABLE_EXPRESSION: {
+				LogicalVariable var = context.getVar(((VariableExpr) expr)
+						.getVar().getId());
+				args.add(new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(var)));
+				break;
+			}
+			case LITERAL_EXPRESSION: {
+				LiteralExpr val = (LiteralExpr) expr;
+				args.add(new MutableObject<ILogicalExpression>(
+						new ConstantExpression(
+								new AsterixConstantValue(ConstantHelper
+										.objectFromLiteral(val.getValue())))));
+				break;
+			}
+			default: {
+				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+						expr, topOp);
+				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
+						.getValue();
+				args.add(new MutableObject<ILogicalExpression>(eo.first));
+				if (o1 != null
+						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
+								o1, topOp))) {
+					topOp = eo.second;
+				}
+				break;
+			}
+			}
+		}
 
-        for (Expression expr : fcall.getExprList()) {
-            switch (expr.getKind()) {
-                case VARIABLE_EXPRESSION: {
-                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
-                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
-                    break;
-                }
-                case LITERAL_EXPRESSION: {
-                    LiteralExpr val = (LiteralExpr) expr;
-                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                            ConstantHelper.objectFromLiteral(val.getValue())))));
-                    break;
-                }
-                default: {
-                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
-                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
-                    args.add(new MutableObject<ILogicalExpression>(eo.first));
-                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
-                        topOp = eo.second;
-                    }
-                    break;
-                }
-            }
-        }
+		FunctionIdentifier fi = new FunctionIdentifier(
+				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+		FunctionIdentifier builtinAquafi = afi == null ? null : afi
+				.getFunctionIdentifier();
 
-        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 {
+			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+					fid.getFunctionName());
+			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
+					.getBuiltinFunctionIdentifier(fi);
+			if (builtinAsterixFi != null) {
+				fi = builtinAsterixFi;
+			}
+		}
+		AbstractFunctionCallExpression f;
+		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
+					args);
+		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+					FunctionUtils.getFunctionInfo(fi), args);
+			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
+					.returnsUniqueValues(fi));
+			f = ufce;
+		} else {
+			f = new ScalarFunctionCallExpression(
+					FunctionUtils.getFunctionInfo(fi), args);
+		}
+		AssignOperator op = new AssignOperator(v,
+				new MutableObject<ILogicalExpression>(f));
+		if (topOp != null) {
+			op.getInputs().add(topOp);
+		}
 
+		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+	}
 
-        if (builtinAquafi != null) {
-            fi = builtinAquafi;
-        } else {
-            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName(), false);
-            FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
-            if (builtinAsterixFi != null) {
-                fi = builtinAsterixFi;
-            }
-        }
-        AbstractFunctionCallExpression f;
-        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
-        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(fi), args);
-            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
-            f = ufce;
-        } else {
-            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
-        }
-        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
-        if (topOp != null) {
-            op.getInputs().add(topOp);
-        }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
+			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
+		// TODO Auto-generated method stub
+		throw new NotImplementedException();
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
+			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		GroupByOperator gOp = new GroupByOperator();
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+			LogicalVariable v;
+			VariableExpr vexpr = ve.getVar();
+			if (vexpr != null) {
+				v = context.newVar(vexpr);
+			} else {
+				v = context.newVar();
+			}
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					ve.getExpr(), topOp);
+			gOp.addGbyExpression(v, eo.first);
+			topOp = eo.second;
+		}
+		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+			LogicalVariable v;
+			VariableExpr vexpr = ve.getVar();
+			if (vexpr != null) {
+				v = context.newVar(vexpr);
+			} else {
+				v = context.newVar();
+			}
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					ve.getExpr(), topOp);
+			gOp.addDecorExpression(v, eo.first);
+			topOp = eo.second;
+		}
+		gOp.getInputs().add(topOp);
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
-        // TODO Auto-generated method stub
-        throw new NotImplementedException();
-    }
+		for (VariableExpr var : gc.getWithVarList()) {
+			LogicalVariable aggVar = context.newVar();
+			LogicalVariable oldVar = context.getVar(var);
+			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			flArgs.add(new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(oldVar)));
+			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
+					.makeAggregateFunctionExpression(
+							AsterixBuiltinFunctions.LISTIFY, flArgs);
+			AggregateOperator agg = new AggregateOperator(
+					mkSingletonArrayList(aggVar),
+					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
+							fListify)));
+			agg.getInputs().add(
+					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
+			// the one bound by the aggregation op.
+			context.setVar(var, aggVar);
+		}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        GroupByOperator gOp = new GroupByOperator();
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-            LogicalVariable v;
-            VariableExpr vexpr = ve.getVar();
-            if (vexpr != null) {
-                v = context.newVar(vexpr);
-            } else {
-                v = context.newVar();
-            }
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
-            gOp.addGbyExpression(v, eo.first);
-            topOp = eo.second;
-        }
-        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-            LogicalVariable v;
-            VariableExpr vexpr = ve.getVar();
-            if (vexpr != null) {
-                v = context.newVar(vexpr);
-            } else {
-                v = context.newVar();
-            }
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
-            gOp.addDecorExpression(v, eo.first);
-            topOp = eo.second;
-        }
-        gOp.getInputs().add(topOp);
+		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
+				gc.hasHashGroupByHint());
+		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+	}
 
-        for (VariableExpr var : gc.getWithVarList()) {
-            LogicalVariable aggVar = context.newVar();
-            LogicalVariable oldVar = context.getVar(var);
-            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
-            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
-            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                    AsterixBuiltinFunctions.LISTIFY, flArgs);
-            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
-                    (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
-            agg.getInputs().add(
-                    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
-            // the one bound by the aggregation op.
-            context.setVar(var, aggVar);
-        }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		// In the most general case, IfThenElse is translated in the following
+		// way.
+		//
+		// We assign the result of the condition to one variable varCond.
+		// We create one subplan which contains the plan for the "then" branch,
+		// on top of which there is a selection whose condition is varCond.
+		// Similarly, we create one subplan for the "else" branch, in which the
+		// selection is not(varCond).
+		// Finally, we concatenate the results. (??)
 
-        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
-        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-    }
+		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
+				.accept(this, tupSource);
+		ILogicalOperator opCond = pCond.first;
+		LogicalVariable varCond = pCond.second;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        // In the most general case, IfThenElse is translated in the following
-        // way.
-        //
-        // We assign the result of the condition to one variable varCond.
-        // We create one subplan which contains the plan for the "then" branch,
-        // on top of which there is a selection whose condition is varCond.
-        // Similarly, we create one subplan for the "else" branch, in which the
-        // selection is not(varCond).
-        // Finally, we concatenate the results. (??)
+		SubplanOperator sp = new SubplanOperator();
+		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
+				new NestedTupleSourceOperator(
+						new MutableObject<ILogicalOperator>(sp)));
 
-        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
-        ILogicalOperator opCond = pCond.first;
-        LogicalVariable varCond = pCond.second;
+		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
+				.accept(this, nestedSource);
+		SelectOperator sel1 = new SelectOperator(
+				new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(varCond)));
+		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-        SubplanOperator sp = new SubplanOperator();
-        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
-                new MutableObject<ILogicalOperator>(sp)));
+		Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr()
+				.accept(this, nestedSource);
+		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+				new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(varCond)));
+		SelectOperator sel2 = new SelectOperator(
+				new MutableObject<ILogicalExpression>(notVarCond));
+		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
-        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(
-                new VariableReferenceExpression(varCond)));
-        sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+		ILogicalPlan p1 = new ALogicalPlanImpl(
+				new MutableObject<ILogicalOperator>(sel1));
+		sp.getNestedPlans().add(p1);
+		ILogicalPlan p2 = new ALogicalPlanImpl(
+				new MutableObject<ILogicalOperator>(sel2));
+		sp.getNestedPlans().add(p2);
 
-        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
-        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(varCond)));
-        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
-        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
+				opCond);
+		sp.getInputs().add(opCondRef);
 
-        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
-        sp.getNestedPlans().add(p1);
-        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
-        sp.getNestedPlans().add(p2);
+		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)));
+		AssignOperator a = new AssignOperator(resV,
+				new MutableObject<ILogicalExpression>(concatNonNull));
+		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
-        sp.getInputs().add(opCondRef);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+	}
 
-        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)));
-        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
-        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
+			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+		LogicalVariable var = context.newVar();
+		AssignOperator a = new AssignOperator(var,
+				new MutableObject<ILogicalExpression>(new ConstantExpression(
+						new AsterixConstantValue(ConstantHelper
+								.objectFromLiteral(l.getValue())))));
+		if (tupSource != null) {
+			a.getInputs().add(tupSource);
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+	}
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
+			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		ArrayList<OperatorType> ops = op.getOpList();
+		int nOps = ops.size();
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-        LogicalVariable var = context.newVar();
-        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
-                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
-        if (tupSource != null) {
-            a.getInputs().add(tupSource);
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-    }
+		if (nOps > 0
+				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+			return visitAndOrOperator(op, tupSource);
+		}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        ArrayList<OperatorType> ops = op.getOpList();
-        int nOps = ops.size();
+		ArrayList<Expression> exprs = op.getExprList();
 
-        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-            return visitAndOrOperator(op, tupSource);
-        }
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-        ArrayList<Expression> exprs = op.getExprList();
+		ILogicalExpression currExpr = null;
+		for (int i = 0; i <= nOps; i++) {
 
-        Mutable<ILogicalOperator> topOp = tupSource;
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					exprs.get(i), topOp);
+			topOp = p.second;
+			ILogicalExpression e = p.first;
+			// now look at the operator
+			if (i < nOps) {
+				if (OperatorExpr.opIsComparison(ops.get(i))) {
+					AbstractFunctionCallExpression c = createComparisonExpression(ops
+							.get(i));
 
-        ILogicalExpression currExpr = null;
-        for (int i = 0; i <= nOps; i++) {
+					// chain the operators
+					if (i == 0) {
+						c.getArguments().add(
+								new MutableObject<ILogicalExpression>(e));
+						currExpr = c;
+						if (op.isBroadcastOperand(i)) {
+							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+							bcast.setObject(BroadcastSide.LEFT);
+							c.getAnnotations()
+									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+											bcast);
+						}
+					} else {
+						((AbstractFunctionCallExpression) currExpr)
+								.getArguments()
+								.add(new MutableObject<ILogicalExpression>(e));
+						c.getArguments()
+								.add(new MutableObject<ILogicalExpression>(
+										currExpr));
+						currExpr = c;
+						if (i == 1 && op.isBroadcastOperand(i)) {
+							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+							bcast.setObject(BroadcastSide.RIGHT);
+							c.getAnnotations()
+									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+											bcast);
+						}
+					}
+				} else {
+					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
+							.get(i));
 
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
-            topOp = p.second;
-            ILogicalExpression e = p.first;
-            // now look at the operator
-            if (i < nOps) {
-                if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
+					if (i == 0) {
+						f.getArguments().add(
+								new MutableObject<ILogicalExpression>(e));
+						currExpr = f;
+					} else {
+						((AbstractFunctionCallExpression) currExpr)
+								.getArguments()
+								.add(new MutableObject<ILogicalExpression>(e));
+						f.getArguments()
+								.add(new MutableObject<ILogicalExpression>(
+										currExpr));
+						currExpr = f;
+					}
+				}
+			} else { // don't forget the last expression...
+				((AbstractFunctionCallExpression) currExpr).getArguments().add(
+						new MutableObject<ILogicalExpression>(e));
+				if (i == 1 && op.isBroadcastOperand(i)) {
+					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+					bcast.setObject(BroadcastSide.RIGHT);
+					((AbstractFunctionCallExpression) currExpr)
+							.getAnnotations()
+							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
+									bcast);
+				}
+			}
+		}
 
-                    // chain the operators
-                    if (i == 0) {
-                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
-                        currExpr = c;
-                        if (op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.LEFT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
-                                new MutableObject<ILogicalExpression>(e));
-                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
-                        currExpr = c;
-                        if (i == 1 && op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.RIGHT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    }
-                } else {
-                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
+		LogicalVariable assignedVar = context.newVar();
+		AssignOperator a = new AssignOperator(assignedVar,
+				new MutableObject<ILogicalExpression>(currExpr));
 
-                    if (i == 0) {
-                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
-                        currExpr = f;
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
-                                new MutableObject<ILogicalExpression>(e));
-                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
-                        currExpr = f;
-                    }
-                }
-            } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<ILogicalExpression>(e));
-                if (i == 1 && op.isBroadcastOperand(i)) {
-                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                    bcast.setObject(BroadcastSide.RIGHT);
-                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
-                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                }
-            }
-        }
+		a.getInputs().add(topOp);
 
-        LogicalVariable assignedVar = context.newVar();
-        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
+		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+	}
 
-        a.getInputs().add(topOp);
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
+			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-    }
+		OrderOperator ord = new OrderOperator();
+		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression e : oc.getOrderbyList()) {
+			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)));
+			topOp = p.second;
+		}
+		ord.getInputs().add(topOp);
+		if (oc.getNumTuples() > 0) {
+			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
+					oc.getNumTuples());
+		}
+		if (oc.getNumFrames() > 0) {
+			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
+					oc.getNumFrames());
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
+			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-        OrderOperator ord = new OrderOperator();
-        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (Expression e : oc.getOrderbyList()) {
-            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)));
-            topOp = p.second;
-        }
-        ord.getInputs().add(topOp);
-        if (oc.getNumTuples() > 0) {
-            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
-        }
-        if (oc.getNumFrames() > 0) {
-            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-    }
+		ILogicalOperator firstOp = null;
+		Mutable<ILogicalOperator> lastOp = null;
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> topOp = tupSource;
+		for (QuantifiedPair qt : qe.getQuantifiedList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
+					qt.getExpr(), topOp);
+			topOp = eo1.second;
+			LogicalVariable uVar = context.newVar(qt.getVarExpr());
+			ILogicalOperator u = new UnnestOperator(uVar,
+					new MutableObject<ILogicalExpression>(
+							makeUnnestExpression(eo1.first)));
 
-        ILogicalOperator firstOp = null;
-        Mutable<ILogicalOperator> lastOp = null;
+			if (firstOp == null) {
+				firstOp = u;
+			}
+			if (lastOp != null) {
+				u.getInputs().add(lastOp);
+			}
+			lastOp = new MutableObject<ILogicalOperator>(u);
+		}
 
-        for (QuantifiedPair qt : qe.getQuantifiedList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
-            topOp = eo1.second;
-            LogicalVariable uVar = context.newVar(qt.getVarExpr());
-            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
-                    makeUnnestExpression(eo1.first)));
+		// We make all the unnest correspond. to quantif. vars. sit on top
+		// in the hope of enabling joins & other optimiz.
+		firstOp.getInputs().add(topOp);
+		topOp = lastOp;
 
-            if (firstOp == null) {
-                firstOp = u;
-            }
-            if (lastOp != null) {
-                u.getInputs().add(lastOp);
-            }
-            lastOp = new MutableObject<ILogicalOperator>(u);
-        }
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
+				qe.getSatisfiesExpr(), topOp);
 
-        // We make all the unnest correspond. to quantif. vars. sit on top
-        // in the hope of enabling joins & other optimiz.
-        firstOp.getInputs().add(topOp);
-        topOp = lastOp;
+		AggregateFunctionCallExpression fAgg;
+		SelectOperator s;
+		if (qe.getQuantifier() == Quantifier.SOME) {
+			s = new SelectOperator(new MutableObject<ILogicalExpression>(
+					eo2.first));
+			s.getInputs().add(eo2.second);
+			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+					new ArrayList<Mutable<ILogicalExpression>>());
+		} else { // EVERY
+			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
+					1);
+			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+			s = new SelectOperator(new MutableObject<ILogicalExpression>(
+					new ScalarFunctionCallExpression(FunctionUtils
+							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+							satExprList)));
+			s.getInputs().add(eo2.second);
+			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+					AsterixBuiltinFunctions.EMPTY_STREAM,
+					new ArrayList<Mutable<ILogicalExpression>>());
+		}
+		LogicalVariable qeVar = context.newVar();
+		AggregateOperator a = new AggregateOperator(
+				mkSingletonArrayList(qeVar),
+				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
+						fAgg)));
+		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+	}
 
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		return q.getBody().accept(this, tupSource);
+	}
 
-        AggregateFunctionCallExpression fAgg;
-        SelectOperator s;
-        if (qe.getQuantifier() == Quantifier.SOME) {
-            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
-            s.getInputs().add(eo2.second);
-            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-                    new ArrayList<Mutable<ILogicalExpression>>());
-        } else { // EVERY
-            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
-            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
-            s.getInputs().add(eo2.second);
-            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
-                    new ArrayList<Mutable<ILogicalExpression>>());
-        }
-        LogicalVariable qeVar = context.newVar();
-        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
-        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
+			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+				FunctionUtils
+						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a = new AssignOperator(v1,
+				new MutableObject<ILogicalExpression>(f));
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (FieldBinding fb : rc.getFbList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
+					fb.getLeftExpr(), topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo1.first));
+			topOp = eo1.second;
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
+					fb.getRightExpr(), topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo2.first));
+			topOp = eo2.second;
+		}
+		a.getInputs().add(topOp);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        return q.getBody().accept(this, tupSource);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
+			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(fid));
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a = new AssignOperator(v1,
+				new MutableObject<ILogicalExpression>(f));
+		Mutable<ILogicalOperator> topOp = tupSource;
+		for (Expression expr : lc.getExprList()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+					expr, topOp);
+			f.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo.first));
+			topOp = eo.second;
+		}
+		a.getInputs().add(topOp);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (FieldBinding fb : rc.getFbList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
-            topOp = eo1.second;
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
-            topOp = eo2.second;
-        }
-        a.getInputs().add(topOp);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
-
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
-        Mutable<ILogicalOperator> topOp = tupSource;
-        for (Expression expr : lc.getExprList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
-            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
-            topOp = eo.second;
-        }
-        a.getInputs().add(topOp);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
-
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Expression expr = u.getExpr();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
-        LogicalVariable v1 = context.newVar();
-        AssignOperator a;
-        if (u.getSign() == Sign.POSITIVE) {
-            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
-        } else {
-            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
-            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-        }
-        a.getInputs().add(eo.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		Expression expr = u.getExpr();
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
+				expr, tupSource);
+		LogicalVariable v1 = context.newVar();
+		AssignOperator a;
+		if (u.getSign() == Sign.POSITIVE) {
+			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
+					eo.first));
+		} else {
+			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+			m.getArguments().add(
+					new MutableObject<ILogicalExpression>(eo.first));
+			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+		}
+		a.getInputs().add(eo.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-        // 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)));
-        a.getInputs().add(tupSource);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
+			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+		// 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)));
+		a.getInputs().add(tupSource);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
-        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
-        s.getInputs().add(p.second);
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
+			WhereClause w, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+				w.getWhereExpr(), tupSource);
+		SelectOperator s = new SelectOperator(
+				new MutableObject<ILogicalExpression>(p.first));
+		s.getInputs().add(p.second);
 
-        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-    }
+		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
-        LimitOperator opLim;
-        Expression offset = lc.getOffset();
-        if (offset != null) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
-            opLim = new LimitOperator(p1.first, p2.first);
-            opLim.getInputs().add(p2.second);
-        } else {
-            opLim = new LimitOperator(p1.first);
-            opLim.getInputs().add(p1.second);
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
+			LimitClause lc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
+				lc.getLimitExpr(), tupSource);
+		LimitOperator opLim;
+		Expression offset = lc.getOffset();
+		if (offset != null) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
+					offset, p1.second);
+			opLim = new LimitOperator(p1.first, p2.first);
+			opLim.getInputs().add(p2.second);
+		} else {
+			opLim = new LimitOperator(p1.first);
+			opLim.getInputs().add(p1.second);
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
-        DieOperator opDie = new DieOperator(p1.first);
-        opDie.getInputs().add(p1.second);
-        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
+			Mutable<ILogicalOperator> tupSource) throws AsterixException {
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
+				lc.getDieExpr(), tupSource);
+		DieOperator opDie = new DieOperator(p1.first);
+		opDie.getInputs().add(p1.second);
+		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-        Mutable<ILogicalOperator> input = null;
-        for (Expression expr : dc.getDistinctByExpr()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
-            exprList.add(new MutableObject<ILogicalExpression>(p.first));
-            input = p.second;
-        }
-        DistinctOperator opDistinct = new DistinctOperator(exprList);
-        opDistinct.getInputs().add(input);
-        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
+			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+		Mutable<ILogicalOperator> input = null;
+		for (Expression expr : dc.getDistinctByExpr()) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					expr, tupSource);
+			exprList.add(new MutableObject<ILogicalExpression>(p.first));
+			input = p.second;
+		}
+		DistinctOperator opDistinct = new DistinctOperator(exprList);
+		opDistinct.getInputs().add(input);
+		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        Mutable<ILogicalOperator> ts = tupSource;
-        ILogicalOperator lastOp = null;
-        LogicalVariable lastVar = null;
-        boolean first = true;
-        for (Expression e : unionExpr.getExprs()) {
-            if (first) {
-                first = false;
-            } else {
-                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
-            }
-            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-            if (lastOp == null) {
-                lastOp = p1.first;
-                lastVar = p1.second;
-            } else {
-                LogicalVariable unnestVar1 = context.newVar();
-                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
-                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
-                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-                LogicalVariable unnestVar2 = context.newVar();
-                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
-                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
-                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
-                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-                        1);
-                LogicalVariable resultVar = context.newVar();
-                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-                        unnestVar1, unnestVar2, resultVar);
-                varMap.add(triple);
-                UnionAllOperator unionOp = new UnionAllOperator(varMap);
-                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
-                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
-                lastVar = resultVar;
-                lastOp = unionOp;
-            }
-        }
-        LogicalVariable aggVar = context.newVar();
-        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
-        aggregVars.add(aggVar);
-        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
-        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                AsterixBuiltinFunctions.LISTIFY, afcExprs);
-        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
+			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		Mutable<ILogicalOperator> ts = tupSource;
+		ILogicalOperator lastOp = null;
+		LogicalVariable lastVar = null;
+		boolean first = true;
+		for (Expression e : unionExpr.getExprs()) {
+			if (first) {
+				first = false;
+			} else {
+				ts = new MutableObject<ILogicalOperator>(
+						new EmptyTupleSourceOperator());
+			}
+			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+			if (lastOp == null) {
+				lastOp = p1.first;
+				lastVar = p1.second;
+			} else {
+				LogicalVariable unnestVar1 = context.newVar();
+				UnnestOperator unnest1 = new UnnestOperator(
+						unnestVar1,
+						new MutableObject<ILogicalExpression>(
+								makeUnnestExpression(new VariableReferenceExpression(
+										lastVar))));
+				unnest1.getInputs().add(
+						new MutableObject<ILogicalOperator>(lastOp));
+				LogicalVariable unnestVar2 = context.newVar();
+				UnnestOperator unnest2 = new UnnestOperator(
+						unnestVar2,
+						new MutableObject<ILogicalExpression>(
+								makeUnnestExpression(new VariableReferenceExpression(
+										p1.second))));
+				unnest2.getInputs().add(
+						new MutableObject<ILogicalOperator>(p1.first));
+				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+						1);
+				LogicalVariable resultVar = context.newVar();
+				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+						unnestVar1, unnestVar2, resultVar);
+				varMap.add(triple);
+				UnionAllOperator unionOp = new UnionAllOperator(varMap);
+				unionOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(unnest1));
+				unionOp.getInputs().add(
+						new MutableObject<ILogicalOperator>(unnest2));
+				lastVar = resultVar;
+				lastOp = unionOp;
+			}
+		}
+		LogicalVariable aggVar = context.newVar();
+		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
+				1);
+		aggregVars.add(aggVar);
+		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
+				1);
+		afcExprs.add(new MutableObject<ILogicalExpression>(
+				new VariableReferenceExpression(lastVar)));
+		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
+				.makeAggregateFunctionExpression(
+						AsterixBuiltinFunctions.LISTIFY, afcExprs);
+		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
+				1);
+		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+	}
 
-    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
-        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-        return new ScalarFunctionCallExpression(finfo);
-    }
+	private AbstractFunctionCallExpression createComparisonExpression(
+			OperatorType t) {
+		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+		return new ScalarFunctionCallExpression(finfo);
+	}
 
-    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-        switch (t) {
-            case EQ: {
-                return AlgebricksBuiltinFunctions.EQ;
-            }
-            case NEQ: {
-                return AlgebricksBuiltinFunctions.NEQ;
-            }
-            case GT: {
-                return AlgebricksBuiltinFunctions.GT;
-            }
-            case GE: {
-                return AlgebricksBuiltinFunctions.GE;
-            }
-            case LT: {
-                return AlgebricksBuiltinFunctions.LT;
-            }
-            case LE: {
-                return AlgebricksBuiltinFunctions.LE;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-    }
+	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+		switch (t) {
+		case EQ: {
+			return AlgebricksBuiltinFunctions.EQ;
+		}
+		case NEQ: {
+			return AlgebricksBuiltinFunctions.NEQ;
+		}
+		case GT: {
+			return AlgebricksBuiltinFunctions.GT;
+		}
+		case GE: {
+			return AlgebricksBuiltinFunctions.GE;
+		}
+		case LT: {
+			return AlgebricksBuiltinFunctions.LT;
+		}
+		case LE: {
+			return AlgebricksBuiltinFunctions.LE;
+		}
+		default: {
+			throw new IllegalStateException();
+		}
+		}
+	}
 
-    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
-            throws AsterixException {
+	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
+			OperatorType t) throws AsterixException {
 
-        FunctionIdentifier fid = null;
-        switch (t) {
-            case PLUS: {
-                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-                break;
-            }
-            case MINUS: {
-                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-                break;
-            }
-            case MUL: {
-                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-                break;
-            }
-            case DIV: {
-                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-                break;
-            }
-            case MOD: {
-                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-                break;
-            }
-            case IDIV: {
-                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-                break;
-            }
-            case CARET: {
-                fid = AsterixBuiltinFunctions.CARET;
-                break;
-            }
-            case AND: {
-                fid = AlgebricksBuiltinFunctions.AND;
-                break;
-            }
-            case OR: {
-                fid = AlgebricksBuiltinFunctions.OR;
-                break;
-            }
-            case FUZZY_EQ: {
-                fid = AsterixBuiltinFunctions.FUZZY_EQ;
-                break;
-            }
+		FunctionIdentifier fid = null;
+		switch (t) {
+		case PLUS: {
+			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+			break;
+		}
+		case MINUS: {
+			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+			break;
+		}
+		case MUL: {
+			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+			break;
+		}
+		case DIV: {
+			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+			break;
+		}
+		case MOD: {
+			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+			break;
+		}
+		case IDIV: {
+			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+			break;
+		}
+		case CARET: {
+			fid = AsterixBuiltinFunctions.CARET;
+			break;
+		}
+		case AND: {
+			fid = AlgebricksBuiltinFunctions.AND;
+			break;
+		}
+		case OR: {
+			fid = AlgebricksBuiltinFunctions.OR;
+			break;
+		}
+		case FUZZY_EQ: {
+			fid = AsterixBuiltinFunctions.FUZZY_EQ;
+			break;
+		}
 
-            default: {
-                throw new NotImplementedException("Operator " + t + " is not yet implemented");
-            }
-        }
-        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
-    }
+		default: {
+			throw new NotImplementedException("Operator " + t
+					+ " is not yet implemented");
+		}
+		}
+		return new ScalarFunctionCallExpression(
+				FunctionUtils.getFunctionInfo(fid));
+	}
 
-    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
-        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-        if (inp == null || inp.size() != 1) {
-            return false;
-        }
-        return inp.get(0) == childCandidate;
-    }
+	private static boolean hasOnlyChild(ILogicalOperator parent,
+			Mutable<ILogicalOperator> childCandidate) {
+		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+		if (inp == null || inp.size() != 1) {
+			return false;
+		}
+		return inp.get(0) == childCandidate;
+	}
 
-    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
-            Mutable<ILogicalOperator> topOp) throws AsterixException {
-        switch (expr.getKind()) {
-            case VARIABLE_EXPRESSION: {
-                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
-                        .getVar().getId()));
-                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
-            }
-            case METAVARIABLE_EXPRESSION: {
-                ILogicalExpression le = metaScopeExp.getVariableReferenceExpression(((VariableExpr) expr).getVar());
-                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le, topOp);
-            }
-            case LITERAL_EXPRESSION: {
-                LiteralExpr val = (LiteralExpr) expr;
-                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
-                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
-            }
-            default: {
-                // Mutable<ILogicalExpression> src = new
-                // Mutable<ILogicalExpression>();
-                // Mutable<ILogicalExpression> src = topOp;
-                if (expressionNeedsNoNesting(expr)) {
-                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
-                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
-                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
-                } else {
-                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
+			Expression expr, Mutable<ILogicalOperator> topOp)
+			throws AsterixException {
+		switch (expr.getKind()) {
+		case VARIABLE_EXPRESSION: {
+			VariableReferenceExpression ve = new VariableReferenceExpression(
+					context.getVar(((VariableExpr) expr).getVar().getId()));
+			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
+					topOp);
+		}
+		case METAVARIABLE_EXPRESSION: {
+			ILogicalExpression le = metaScopeExp
+					.getVariableReferenceExpression(((VariableExpr) expr)
+							.getVar());
+			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le,
+					topOp);
+		}
+		case LITERAL_EXPRESSION: {
+			LiteralExpr val = (LiteralExpr) expr;
+			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+					new ConstantExpression(new AsterixConstantValue(
+							ConstantHelper.objectFromLiteral(val.getValue()))),
+					topOp);
+		}
+		default: {
+			// Mutable<ILogicalExpression> src = new
+			// Mutable<ILogicalExpression>();
+			// Mutable<ILogicalExpression> src = topOp;
+			if (expressionNeedsNoNesting(expr)) {
+				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+						topOp);
+				ILogicalExpression exp = ((AssignOperator) p.first)
+						.getExpressions().get(0).getValue();
+				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+						exp, p.first.getInputs().get(0));
+			} else {
+				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
+				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
+						src);
 
-                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-                        // src.setOperator(topOp.getOperator());
-                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
-                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
-                                p.second), top2);
-                    } else {
-                        SubplanOperator s = new SubplanOperator();
-                        s.getInputs().add(topOp);
-                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
-                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
-                        s.setRootOp(planRoot);
-                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
-                                p.second), new MutableObject<ILogicalOperator>(s));
-                    }
-                }
-            }
-        }
+				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+					// src.setOperator(topOp.getOperator());
+					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
+							p.first);
+					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+							new VariableReferenceExpression(p.second), top2);
+				} else {
+					SubplanOperator s = new SubplanOperator();
+					s.getInputs().add(topOp);
+					src.setValue(new NestedTupleSourceOperator(
+							new MutableObject<ILogicalOperator>(s)));
+					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
+							p.first);
+					s.setRootOp(planRoot);
+					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
+							new VariableReferenceExpression(p.second),
+							new MutableObject<ILogicalOperator>(s));
+				}
+			}
+		}
+		}
 
-    }
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
-            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-        if (isTop) {
-            ProjectOperator pr = new ProjectOperator(resVar);
-            pr.getInputs().add(resOpRef);
-            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
+			boolean noForClause, boolean isTop,
+			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+		if (isTop) {
+			ProjectOperator pr = new ProjectOperator(resVar);
+			pr.getInputs().add(resOpRef);
+			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-        } else if (noForClause) {
-            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
-        } else {
-            return aggListify(resVar, resOpRef, false);
-        }
-    }
+		} else if (noForClause) {
+			return new Pair<ILogicalOperator, LogicalVariable>(
+					resOpRef.getValue(), resVar);
+		} else {
+			return aggListify(resVar, resOpRef, false);
+		}
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
-            boolean bProject) {
-        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
-        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)));
-        agg.getInputs().add(opRef);
-        ILogicalOperator res;
-        if (bProject) {
-            ProjectOperator pr = new ProjectOperator(varListified);
-            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-            res = pr;
-        } else {
-            res = agg;
-        }
-        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-    }
+	private Pair<ILogicalOperator, LogicalVariable> aggListify(
+			LogicalVariable var, Mutable<ILogicalOperator> opRef,
+			boolean bProject) {
+		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
+				.makeAggregateFunctionExpression(
+						AsterixBuiltinFunctions.LISTIFY,
+						new ArrayList<Mutable<ILogicalExpression>>());
+		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)));
+		agg.getInputs().add(opRef);
+		ILogicalOperator res;
+		if (bProject) {
+			ProjectOperator pr = new ProjectOperator(varListified);
+			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+			res = pr;
+		} else {
+			res = agg;
+		}
+		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+	}
 
-    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        ArrayList<OperatorType> ops = op.getOpList();
-        int nOps = ops.size();
+	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
+			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		ArrayList<OperatorType> ops = op.getOpList();
+		int nOps = ops.size();
 
-        ArrayList<Expression> exprs = op.getExprList();
+		ArrayList<Expression> exprs = op.getExprList();
 
-        Mutable<ILogicalOperator> topOp = tupSource;
+		Mutable<ILogicalOperator> topOp = tupSource;
 
-        OperatorType opLogical = ops.get(0);
-        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+		OperatorType opLogical = ops.get(0);
+		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-        for (int i = 0; i <= nOps; i++) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
-            topOp = p.second;
-            // now look at the operator
-            if (i < nOps) {
-                if (ops.get(i) != opLogical) {
-                    throw new TranslationException("Unexpected operator " + ops.get(i)
-                            + " in an OperatorExpr starting with " + opLogical);
-                }
-            }
-            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
-        }
+		for (int i = 0; i <= nOps; i++) {
+			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
+					exprs.get(i), topOp);
+			topOp = p.second;
+			// now look at the operator
+			if (i < nOps) {
+				if (ops.get(i) != opLogical) {
+					throw new TranslationException("Unexpected operator "
+							+ ops.get(i) + " in an OperatorExpr starting with "
+							+ opLogical);
+				}
+			}
+			f.getArguments()
+					.add(new MutableObject<ILogicalExpression>(p.first));
+		}
 
-        LogicalVariable assignedVar = context.newVar();
-        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
-        a.getInputs().add(topOp);
+		LogicalVariable assignedVar = context.newVar();
+		AssignOperator a = new AssignOperator(assignedVar,
+				new MutableObject<ILogicalExpression>(f));
+		a.getInputs().add(topOp);
 
-        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-    }
+	}
 
-    private static boolean expressionNeedsNoNesting(Expression expr) {
-        Kind k = expr.getKind();
-        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
-                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
-                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
-    }
+	private static boolean expressionNeedsNoNesting(Expression expr) {
+		Kind k = expr.getKind();
+		return k == Kind.LITERAL_EXPRESSION
+				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
+				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
+				|| k == Kind.OP_EXPRESSION
+				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
+				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
+				|| k == Kind.UNARY_EXPRESSION;
+	}
 
-    private <T> ArrayList<T> mkSingletonArrayList(T item) {
-        ArrayList<T> array = new ArrayList<T>(1);
-        array.add(item);
-        return array;
-    }
+	private <T> ArrayList<T> mkSingletonArrayList(T item) {
+		ArrayList<T> array = new ArrayList<T>(1);
+		array.add(item);
+		return array;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
+			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
+			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
+			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
+			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+			WriteFromQueryResultStatement stmtLoad,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
+			DropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
+			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-  
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
+			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
+			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(MetaVariableClause mc,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc.getVar()), null);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(
+			MetaVariableClause mc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc
+				.getVar()), null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(JoinClause jc, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
-        // Pair<ILogicalOperator, LogicalVariable> leftSide =
-        // jc.getLeftExpr().accept(this, tupSource);
-        Mutable<ILogicalOperator> opRef = tupSource;
-        Pair<ILogicalOperator, LogicalVariable> leftSide = null;
-        for (Clause c : jc.getLeftClauses()) {
-            leftSide = c.accept(this, opRef);
-            opRef = new MutableObject<ILogicalOperator>(leftSide.first);
-        }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(
+			JoinClause jc, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		// Pair<ILogicalOperator, LogicalVariable> leftSide =
+		// jc.getLeftExpr().accept(this, tupSource);
+		Mutable<ILogicalOperator> opRef = tupSource;
+		Pair<ILogicalOperator, LogicalVariable> leftSide = null;
+		for (Clause c : jc.getLeftClauses()) {
+			leftSide = c.accept(this, opRef);
+			opRef = new MutableObject<ILogicalOperator>(leftSide.first);
+		}
 
-        // Pair<ILogicalOperator, LogicalVariable> rightSide =
-        // jc.getRightExpr().accept(this, tupSource);
-        opRef = tupSource;
-        Pair<ILogicalOperator, LogicalVariable> rightSide = null;
-        for (Clause c : jc.getRightClauses()) {
-            rightSide = c.accept(this, opRef);
-            opRef = new MutableObject<ILogicalOperator>(rightSide.first);
-        }
+		// Pair<ILogicalOperator, LogicalVariable> rightSide =
+		// jc.getRightExpr().accept(this, tupSource);
+		opRef = tupSource;
+		Pair<ILogicalOperator, LogicalVariable> rightSide = null;
+		for (Clause c : jc.getRightClauses()) {
+			rightSide = c.accept(this, opRef);
+			opRef = new MutableObject<ILogicalOperator>(rightSide.first);
+		}
 
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(jc.getWhereExpr(),
-                tupSource);
+		Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(
+				jc.getWhereExpr(), tupSource);
 
-        AbstractBinaryJoinOperator join;
-        switch (jc.getKind()) {
-            case INNER: {
-                join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
-                break;
-            }
-            case LEFT_OUTER: {
-                join = new LeftOuterJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
-                break;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-        join.getInputs().add(new MutableObject<ILogicalOperator>(leftSide.first));
-        join.getInputs().add(new MutableObject<ILogicalOperator>(rightSide.first));
-        return new Pair<ILogicalOperator, LogicalVariable>(join, null);
-    }
+		AbstractBinaryJoinOperator join;
+		switch (jc.getKind()) {
+		case INNER: {
+			join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(
+					whereCond.first));
+			break;
+		}
+		case LEFT_OUTER: {
+			join = new LeftOuterJoinOperator(
+					new MutableObject<ILogicalExpression>(whereCond.first));
+			break;
+		}
+		default: {
+			throw new IllegalStateException();
+		}
+		}
+		join.getInputs().add(
+				new MutableObject<ILogicalOperator>(leftSide.first));
+		join.getInputs().add(
+				new MutableObject<ILogicalOperator>(rightSide.first));
+		return new Pair<ILogicalOperator, LogicalVariable>(join, null);
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(MetaVariableExpr me,
-            Mutable<ILogicalOperator> tupSource) throws AsterixException {
-        LogicalVariable var = context.newVar();
-        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
-                metaScopeExp.getVariableReferenceExpression(me.getVar())));
-        a.getInputs().add(tupSource);
-        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(
+			MetaVariableExpr me, Mutable<ILogicalOperator> tupSource)
+			throws AsterixException {
+		LogicalVariable var = context.newVar();
+		AssignOperator a = new AssignOperator(var,
+				new MutableObject<ILogicalExpression>(metaScopeExp
+						.getVariableReferenceExpression(me.getVar())));
+		a.getInputs().add(tupSource);
+		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+	}
 
-    public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
-        metaScopeOp.put(id, op);
-    }
+	public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
+		metaScopeOp.put(id, op);
+	}
 
-    public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
-        metaScopeExp.put(id, var);
-    }
+	public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
+		metaScopeExp.put(id, var);
+	}
 
-    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-        switch (expr.getExpressionTag()) {
-            case VARIABLE: {
-                return new UnnestingFunctionCallExpression(
-                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-                        new MutableObject<ILogicalExpression>(expr));
-            }
-            case FUNCTION_CALL: {
-                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                if (fce.getKind() == FunctionKind.UNNEST) {
-                    return expr;
-                } else {
-                    return new UnnestingFunctionCallExpression(
-                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-                            new MutableObject<ILogicalExpression>(expr));
-                }
-            }
-            default: {
-                return expr;
-            }
-        }
-    }
+	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+		switch (expr.getExpressionTag()) {
+		case VARIABLE: {
+			return new UnnestingFunctionCallExpression(
+					FunctionUtils
+							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+					new MutableObject<ILogicalExpression>(expr));
+		}
+		case FUNCTION_CALL: {
+			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+			if (fce.getKind() == FunctionKind.UNNEST) {
+				return expr;
+			} else {
+				return new UnnestingFunctionCallExpression(
+						FunctionUtils
+								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+						new MutableObject<ILogicalExpression>(expr));
+			}
+		}
+		default: {
+			return expr;
+		}
+		}
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
+			InsertStatement insert, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
+			DeleteStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
+			UpdateStatement update, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
+			UpdateClause del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
+			DataverseDecl dv, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
+			DatasetDecl dd, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
+			SetStatement ss, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
+			WriteStatement ws, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+			WriteFromQueryResultStatement stmtLoad,
+			Mutable<ILogicalOperator> arg) throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
+			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
+			IndexDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
+			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
+			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
+			TypeDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
+			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-            throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visit(
+			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
+			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 
-    @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
-            Mutable<ILogicalOperator> arg) throws AsterixException {
-        // TODO Auto-generated method stub
-        return null;
-    }
+	@Override
+	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
+			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
+			throws AsterixException {
+		// TODO Auto-generated method stub
+		return null;
+	}
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/ConstantHelper.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/ConstantHelper.java
index e010d3c..bf2b64a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/ConstantHelper.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/ConstantHelper.java
@@ -1,21 +1,23 @@
 package edu.uci.ics.asterix.translator;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 import edu.uci.ics.asterix.aql.literal.DoubleLiteral;
 import edu.uci.ics.asterix.aql.literal.FloatLiteral;
 import edu.uci.ics.asterix.aql.literal.IntegerLiteral;
+import edu.uci.ics.asterix.aql.literal.LongIntegerLiteral;
 import edu.uci.ics.asterix.aql.literal.StringLiteral;
 import edu.uci.ics.asterix.om.base.ABoolean;
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.base.IAObject;
 
 public final class ConstantHelper {
 
-    public static IAObject objectFromLiteral(ILiteral valLiteral) {
+    public static IAObject objectFromLiteral(Literal valLiteral) {
         switch (valLiteral.getLiteralType()) {
             case DOUBLE: {
                 DoubleLiteral d = (DoubleLiteral) valLiteral;
@@ -32,6 +34,10 @@
                 IntegerLiteral il = (IntegerLiteral) valLiteral;
                 return new AInt32(il.getValue());
             }
+            case LONG: {
+                LongIntegerLiteral il = (LongIntegerLiteral) valLiteral;
+                return new AInt64(il.getValue());                
+            }
             case NULL: {
                 return ANull.NULL;
             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
index f7d6e85..c8b1079 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
@@ -35,7 +35,6 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
@@ -79,18 +78,20 @@
                 case LOAD_FROM_FILE: {
                     LoadFromFileStatement loadStmt = (LoadFromFileStatement) stmt;
                     CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(loadStmt.getDatasetName()
-                            .getValue(), loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
+                            .getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
+                            loadStmt.dataIsAlreadySorted());
                     dmlStatements.add(cls);
                     // Also load the dataset's secondary indexes.
                     List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
-                    		compiledDeclarations.getDataverseName(), loadStmt.getDatasetName().getValue());
+                            compiledDeclarations.getDataverseName(), loadStmt.getDatasetName().getValue());
                     for (Index index : datasetIndexes) {
                         if (!index.isSecondaryIndex()) {
                             continue;
                         }
                         // Create CompiledCreateIndexStatement from metadata entity 'index'.
                         CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(),
-                                index.getDatasetName(), index.getKeyFieldNames(), index.getIndexType());
+                                index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(),
+                                index.getIndexType());
                         dmlStatements.add(cis);
                     }
                     break;
@@ -112,32 +113,33 @@
                     // 2. If the create index stmt came before the load stmt, then we would first create an empty index only to load it again later. 
                     // This may cause problems because the index would be considered loaded (even though it was loaded empty). 
                     for (Statement s : aqlStatements) {
-                    	if (s.getKind() != Kind.LOAD_FROM_FILE) {
-                    		continue;
-                    	}
-                    	LoadFromFileStatement loadStmt = (LoadFromFileStatement) s;
-                    	if (loadStmt.getDatasetName().equals(cis.getDatasetName())) {
-                    		cis.setNeedToCreate(false);
-                    	}
+                        if (s.getKind() != Kind.LOAD_FROM_FILE) {
+                            continue;
+                        }
+                        LoadFromFileStatement loadStmt = (LoadFromFileStatement) s;
+                        if (loadStmt.getDatasetName().equals(cis.getDatasetName())) {
+                            cis.setNeedToCreate(false);
+                        }
                     }
                     if (cis.getNeedToCreate()) {
                         CompiledCreateIndexStatement ccis = new CompiledCreateIndexStatement(cis.getIndexName()
-                                .getValue(), cis.getDatasetName().getValue(), cis.getFieldExprs(), cis.getIndexType());
+                                .getValue(), cis.getDatasetName().getValue(), cis.getFieldExprs(), cis.getGramLength(),
+                                cis.getIndexType());
                         dmlStatements.add(ccis);
-                    } 
+                    }
                     break;
                 }
                 case INSERT: {
                     InsertStatement is = (InsertStatement) stmt;
-                    CompiledInsertStatement clfrqs = new CompiledInsertStatement(is.getDatasetName().getValue(), is
-                            .getQuery(), is.getVarCounter());
+                    CompiledInsertStatement clfrqs = new CompiledInsertStatement(is.getDatasetName().getValue(),
+                            is.getQuery(), is.getVarCounter());
                     dmlStatements.add(clfrqs);
                     break;
                 }
                 case DELETE: {
                     DeleteStatement ds = (DeleteStatement) stmt;
-                    CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(ds.getVariableExpr(), ds
-                            .getDatasetName(), ds.getCondition(), ds.getDieClause(), ds.getVarCounter(),
+                    CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(ds.getVariableExpr(),
+                            ds.getDatasetName(), ds.getCondition(), ds.getDieClause(), ds.getVarCounter(),
                             compiledDeclarations);
                     dmlStatements.add(clfrqs);
                     break;
@@ -145,8 +147,8 @@
 
                 case BEGIN_FEED: {
                     BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-                    CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(bfs.getDatasetName(), bfs
-                            .getQuery(), bfs.getVarCounter());
+                    CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(bfs.getDatasetName(),
+                            bfs.getQuery(), bfs.getVarCounter());
                     dmlStatements.add(cbfs);
                     Dataset dataset;
                     try {
@@ -167,8 +169,8 @@
 
                 case CONTROL_FEED: {
                     ControlFeedStatement cfs = (ControlFeedStatement) stmt;
-                    CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(), cfs
-                            .getDatasetName(), cfs.getAlterAdapterConfParams());
+                    CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
+                            cfs.getDatasetName(), cfs.getAlterAdapterConfParams());
                     dmlStatements.add(clcfs);
                     break;
 
@@ -184,16 +186,20 @@
     }
 
     public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
-        private String indexName;
-        private String datasetName;
-        private List<String> keyFields;
-        private IndexType indexType;
+        private final String indexName;
+        private final String datasetName;
+        private final List<String> keyFields;
+        private final IndexType indexType;
+
+        // Specific to NGram index.
+        private final int gramLength;
 
         public CompiledCreateIndexStatement(String indexName, String datasetName, List<String> keyFields,
-                IndexType indexType) {
+                int gramLength, IndexType indexType) {
             this.indexName = indexName;
             this.datasetName = datasetName;
             this.keyFields = keyFields;
+            this.gramLength = gramLength;
             this.indexType = indexType;
         }
 
@@ -213,6 +219,10 @@
             return indexType;
         }
 
+        public int getGramLength() {
+            return gramLength;
+        }
+
         @Override
         public Kind getKind() {
             return Kind.CREATE_INDEX;
@@ -450,11 +460,11 @@
                 clauseList.add(dieClause);
             }
 
-            AqlCompiledDatasetDecl aqlDataset = compiledDeclarations.findDataset(datasetName);
-            if (aqlDataset == null) {
+            Dataset dataset = compiledDeclarations.findDataset(datasetName);
+            if (dataset == null) {
                 throw new AlgebricksException("Unknown dataset " + datasetName);
             }
-            String itemTypeName = aqlDataset.getItemTypeName();
+            String itemTypeName = dataset.getItemTypeName();
             IAType itemType = compiledDeclarations.findType(itemTypeName);
             ARecordType recType = (ARecordType) itemType;
             String[] fieldNames = recType.getFieldNames();
diff --git a/asterix-algebra/src/main/javacc/AQLPlus.jj b/asterix-algebra/src/main/javacc/AQLPlus.jj
index 9a8f783..3422652 100644
--- a/asterix-algebra/src/main/javacc/AQLPlus.jj
+++ b/asterix-algebra/src/main/javacc/AQLPlus.jj
@@ -20,8 +20,9 @@
 import edu.uci.ics.asterix.aql.literal.FloatLiteral;
 import edu.uci.ics.asterix.aql.literal.DoubleLiteral;
 import edu.uci.ics.asterix.aql.literal.FalseLiteral;
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 import edu.uci.ics.asterix.aql.literal.IntegerLiteral;
+import edu.uci.ics.asterix.aql.literal.LongIntegerLiteral;
 import edu.uci.ics.asterix.aql.literal.NullLiteral;
 import edu.uci.ics.asterix.aql.literal.StringLiteral;
 import edu.uci.ics.asterix.aql.literal.TrueLiteral;
@@ -778,8 +779,9 @@
 	{
 		if(expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
 		{
-			ILiteral lit = ((LiteralExpr)expr).getValue();
-			if(lit.getLiteralType() == ILiteral.Type.INTEGER) {
+			Literal lit = ((LiteralExpr)expr).getValue();
+			if(lit.getLiteralType() == Literal.Type.INTEGER || 
+ 			   lit.getLiteralType() == Literal.Type.LONG) {
 				idx = Integer.valueOf(lit.getStringValue());
 			}	
 			else {
@@ -809,7 +811,7 @@
   Expression expr = null;
 }
 {
-  //ILiteral | VariableRef | ListConstructor | RecordConstructor | FunctionCallExpr | ParenthesizedExpression
+  //Literal | VariableRef | ListConstructor | RecordConstructor | FunctionCallExpr | ParenthesizedExpression
 	(
 	  expr =Literal() 
 	   | expr = FunctionCallExpr()
@@ -846,7 +848,11 @@
  	 | <INTEGER_LITERAL>
     {
       t= getToken(0);
-	  lit.setValue(new IntegerLiteral(new Integer(t.image)));
+	  try {
+	      lit.setValue(new IntegerLiteral(new Integer(t.image)));
+	  } catch(NumberFormatException ex) {
+	      lit.setValue(new LongIntegerLiteral(new Long(t.image)));
+	  }
 	}
      | < FLOAT_LITERAL >
     {
@@ -1231,12 +1237,9 @@
 	extendCurrentScope();
 }
 {
-    "let" varExp = Variable()
+    "let" varExp = Variable() ":=" beExp = Expression()
     {
       getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
-	}
-     ":=" beExp = Expression()
-    {
       lc.setVarExpr(varExp);
       lc.setBeExpr(beExp);
       return lc;
diff --git a/asterix-app/data/semistructured/co1k_olist/customer.adm b/asterix-app/data/semistructured/co1k_olist/customer.adm
new file mode 100644
index 0000000..dfe6134
--- /dev/null
+++ b/asterix-app/data/semistructured/co1k_olist/customer.adm
@@ -0,0 +1,1000 @@
+{  "cid": 748,  "name": "Petra Ganes",  "interests": [  ],  "children": [ {  "name": "Perry Ganes" }, {  "name": "Krista Ganes",  "age": 54 }, {  "name": "Kayce Ganes",  "age": 52 }, {  "name": "Eleni Ganes" } ] }
+{  "cid": 871,  "name": "Lona Dacus",  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Pablo Dacus" }, {  "name": "Darlene Dacus",  "age": 45 }, {  "name": "Darius Dacus",  "age": 31 }, {  "name": "Cordia Dacus" } ] }
+{  "cid": 808,  "name": "Brande Decius",  "interests": [ "Basketball", "Fishing", "Puzzles" ],  "children": [ {  "name": "Li Decius",  "age": 56 }, {  "name": "Eusebio Decius",  "age": 50 }, {  "name": "Clementina Decius",  "age": 29 } ] }
+{  "cid": 276,  "name": "Denyse Groth",  "age": 81,  "address": {  "number": 6825,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Databases", "Fishing", "Movies" ],  "children": [ {  "name": "Marilee Groth",  "age": 12 }, {  "name": "Lyla Groth",  "age": 46 }, {  "name": "Sarah Groth" } ] }
+{  "cid": 988,  "name": "Dagmar Plasky",  "age": 89,  "address": {  "number": 1219,  "street": "Park St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Dann Plasky",  "age": 59 }, {  "name": "Raye Plasky" }, {  "name": "Sammie Plasky",  "age": 36 }, {  "name": "Kasi Plasky",  "age": 24 } ] }
+{  "cid": 909,  "name": "Mariko Sharar",  "interests": [ "Squash", "Movies", "Computers" ],  "children": [  ] }
+{  "cid": 233,  "name": "Sammy Coalter",  "interests": [ "Fishing", "Base Jumping" ],  "children": [ {  "name": "Twana Coalter" }, {  "name": "Nenita Coalter",  "age": 30 } ] }
+{  "cid": 896,  "name": "Georgina Even",  "interests": [ "Music", "Databases", "Base Jumping", "Cigars" ],  "children": [ {  "name": "Angelica Even",  "age": 25 } ] }
+{  "cid": 772,  "name": "Shan Renney",  "interests": [ "Books", "Books", "Bass", "Cooking" ],  "children": [ {  "name": "Bessie Renney",  "age": 32 }, {  "name": "Dionna Renney",  "age": 46 }, {  "name": "Vonda Renney" }, {  "name": "Pamella Renney",  "age": 16 } ] }
+{  "cid": 723,  "name": "Teressa Krol",  "age": 22,  "address": {  "number": 8036,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Music" ],  "children": [ {  "name": "Tuan Krol" }, {  "name": "Judi Krol" }, {  "name": "Maddie Krol" } ] }
+{  "cid": 43,  "name": "Rina Bonyai",  "age": 77,  "address": {  "number": 3640,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Tennis", "Walking", "Computers", "Books" ],  "children": [ {  "name": "Mirta Bonyai",  "age": 51 }, {  "name": "Terrance Bonyai" }, {  "name": "Maria Bonyai",  "age": 51 }, {  "name": "Dulcie Bonyai" } ] }
+{  "cid": 816,  "name": "Cheyenne Eddie",  "interests": [ "Walking", "Cooking" ],  "children": [ {  "name": "Kathe Eddie" }, {  "name": "Charles Eddie" } ] }
+{  "cid": 302,  "name": "Rosalie Laderer",  "interests": [ "Tennis", "Movies", "Movies" ],  "children": [ {  "name": "Moriah Laderer" }, {  "name": "Liana Laderer",  "age": 21 }, {  "name": "Genia Laderer",  "age": 45 } ] }
+{  "cid": 686,  "name": "Trudi Arnette",  "interests": [  ],  "children": [ {  "name": "Adrian Arnette",  "age": 43 }, {  "name": "Hulda Arnette",  "age": 34 }, {  "name": "Shamika Arnette" } ] }
+{  "cid": 565,  "name": "Shantell Rima",  "age": 82,  "address": {  "number": 205,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Boyce Rima",  "age": 67 }, {  "name": "Woodrow Rima",  "age": 18 }, {  "name": "Helene Rima" }, {  "name": "David Rima" } ] }
+{  "cid": 334,  "name": "Valarie Tattershall",  "interests": [ "Books", "Walking", "Skiing", "Movies" ],  "children": [  ] }
+{  "cid": 715,  "name": "Zoraida Scribner",  "interests": [  ],  "children": [ {  "name": "Ninfa Scribner",  "age": 31 } ] }
+{  "cid": 963,  "name": "Mila Ditmars",  "age": 29,  "address": {  "number": 5850,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [ "Music" ],  "children": [  ] }
+{  "cid": 966,  "name": "Brigitte Quimby",  "age": 13,  "address": {  "number": 203,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Skiing", "Tennis" ],  "children": [ {  "name": "Ilona Quimby" }, {  "name": "Shaunte Quimby" }, {  "name": "Lorie Quimby" } ] }
+{  "cid": 826,  "name": "Ressie Feenstra",  "interests": [  ],  "children": [ {  "name": "Sasha Feenstra" } ] }
+{  "cid": 238,  "name": "Marcelina Redic",  "interests": [ "Cigars", "Cigars", "Coffee" ],  "children": [ {  "name": "Renate Redic" }, {  "name": "Kyoko Redic" }, {  "name": "Dorthey Redic" } ] }
+{  "cid": 454,  "name": "Irving Lhuillier",  "interests": [  ],  "children": [ {  "name": "Emile Lhuillier" }, {  "name": "Albert Lhuillier" }, {  "name": "Ingeborg Lhuillier",  "age": 23 }, {  "name": "Shila Lhuillier",  "age": 55 } ] }
+{  "cid": 537,  "name": "Mara Hugar",  "interests": [ "Fishing", "Skiing", "Skiing" ],  "children": [ {  "name": "Krista Hugar" } ] }
+{  "cid": 794,  "name": "Annabel Leins",  "age": 75,  "address": {  "number": 9761,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Bass", "Computers", "Bass", "Cigars" ],  "children": [ {  "name": "Oswaldo Leins",  "age": 21 } ] }
+{  "cid": 483,  "name": "Elsa Vigen",  "interests": [ "Wine", "Databases" ],  "children": [ {  "name": "Larae Vigen" }, {  "name": "Elwood Vigen" } ] }
+{  "cid": 746,  "name": "Rosalinda Pola",  "interests": [ "Cooking", "Computers", "Walking", "Cigars" ],  "children": [ {  "name": "Maribel Pola",  "age": 19 }, {  "name": "Chaya Pola" }, {  "name": "Shauna Pola" }, {  "name": "Elenora Pola",  "age": 22 } ] }
+{  "cid": 559,  "name": "Carolyne Shiroma",  "interests": [ "Movies", "Running" ],  "children": [ {  "name": "Ying Shiroma",  "age": 57 } ] }
+{  "cid": 9,  "name": "Dreama Nuccio",  "age": 55,  "address": {  "number": 95,  "street": "Main St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Ricardo Nuccio",  "age": 28 }, {  "name": "See Nuccio",  "age": 34 } ] }
+{  "cid": 844,  "name": "Madelene Ten",  "interests": [ "Squash" ],  "children": [ {  "name": "Johanne Ten",  "age": 39 }, {  "name": "Lurline Ten" }, {  "name": "Cathy Ten",  "age": 49 } ] }
+{  "cid": 526,  "name": "Catrice Swantak",  "interests": [ "Music", "Cigars", "Base Jumping", "Wine" ],  "children": [ {  "name": "Eun Swantak" }, {  "name": "Waylon Swantak" }, {  "name": "Carroll Swantak" } ] }
+{  "cid": 616,  "name": "Shanda Dussault",  "interests": [  ],  "children": [ {  "name": "Darrick Dussault" } ] }
+{  "cid": 217,  "name": "Scott Fulks",  "interests": [ "Computers" ],  "children": [  ] }
+{  "cid": 864,  "name": "Katharyn Zanotti",  "age": 62,  "address": {  "number": 8336,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Magan Zanotti" }, {  "name": "Jacinto Zanotti" } ] }
+{  "cid": 902,  "name": "Tajuana Foote",  "interests": [ "Walking", "Cooking", "Squash", "Tennis" ],  "children": [ {  "name": "Lesia Foote",  "age": 14 }, {  "name": "Rene Foote",  "age": 11 }, {  "name": "Meryl Foote" }, {  "name": "Vanetta Foote" } ] }
+{  "cid": 964,  "name": "Stephany Soders",  "interests": [ "Tennis", "Wine", "Computers" ],  "children": [  ] }
+{  "cid": 888,  "name": "Natalie Nocella",  "age": 66,  "address": {  "number": 2856,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Noel Nocella",  "age": 26 }, {  "name": "Damon Nocella",  "age": 29 }, {  "name": "Joesph Nocella",  "age": 33 }, {  "name": "Nidia Nocella" } ] }
+{  "cid": 991,  "name": "Leonel Toepperwein",  "age": 62,  "address": {  "number": 8356,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Coffee", "Books" ],  "children": [ {  "name": "Sean Toepperwein" }, {  "name": "Charline Toepperwein",  "age": 49 }, {  "name": "Hattie Toepperwein",  "age": 22 }, {  "name": "Melida Toepperwein" } ] }
+{  "cid": 619,  "name": "Luanne Elmquist",  "interests": [  ],  "children": [ {  "name": "Burton Elmquist",  "age": 11 }, {  "name": "Melvin Elmquist" } ] }
+{  "cid": 20,  "name": "Annice Fulwider",  "age": 59,  "address": {  "number": 4257,  "street": "Park St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Arica Fulwider",  "age": 47 }, {  "name": "Charlotte Fulwider",  "age": 16 }, {  "name": "Robbi Fulwider",  "age": 29 } ] }
+{  "cid": 905,  "name": "Pandora Azzarella",  "interests": [  ],  "children": [ {  "name": "Lane Azzarella" }, {  "name": "Joi Azzarella",  "age": 19 } ] }
+{  "cid": 839,  "name": "Annetta Bertsche",  "age": 31,  "address": {  "number": 5823,  "street": "Hill St.",  "city": "Portland" },  "interests": [ "Music", "Coffee", "Cigars", "Computers" ],  "children": [ {  "name": "Annita Bertsche" }, {  "name": "Violette Bertsche",  "age": 13 }, {  "name": "An Bertsche" } ] }
+{  "cid": 873,  "name": "Artie Gongalves",  "age": 74,  "address": {  "number": 584,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Basketball", "Databases", "Puzzles", "Skiing" ],  "children": [ {  "name": "Chester Gongalves",  "age": 10 } ] }
+{  "cid": 456,  "name": "Kim Cervera",  "age": 89,  "address": {  "number": 3967,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Winona Cervera",  "age": 37 }, {  "name": "Shanice Cervera" }, {  "name": "Michaele Cervera" } ] }
+{  "cid": 666,  "name": "Pamila Burzlaff",  "age": 68,  "address": {  "number": 6543,  "street": "View St.",  "city": "Portland" },  "interests": [ "Squash", "Cigars", "Movies" ],  "children": [  ] }
+{  "cid": 64,  "name": "Victor Susor",  "age": 32,  "address": {  "number": 1690,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Running", "Computers" ],  "children": [  ] }
+{  "cid": 187,  "name": "Seema Hartsch",  "age": 80,  "address": {  "number": 6629,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Coffee", "Coffee", "Cigars" ],  "children": [ {  "name": "Suellen Hartsch" }, {  "name": "Pennie Hartsch",  "age": 20 }, {  "name": "Aubrey Hartsch" }, {  "name": "Randy Hartsch",  "age": 32 } ] }
+{  "cid": 771,  "name": "Marisela Tredo",  "interests": [ "Tennis", "Coffee" ],  "children": [ {  "name": "Ardell Tredo",  "age": 21 }, {  "name": "Evelynn Tredo",  "age": 16 } ] }
+{  "cid": 859,  "name": "Mozelle Catillo",  "age": 61,  "address": {  "number": 253,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Cooking", "Wine" ],  "children": [  ] }
+{  "cid": 609,  "name": "Mindi Dieudonne",  "interests": [ "Puzzles" ],  "children": [  ] }
+{  "cid": 521,  "name": "Frankie Hofmann",  "interests": [ "Databases", "Movies" ],  "children": [ {  "name": "Shirlee Hofmann",  "age": 32 }, {  "name": "Jacque Hofmann",  "age": 23 }, {  "name": "Jazmin Hofmann" }, {  "name": "Serena Hofmann",  "age": 56 } ] }
+{  "cid": 994,  "name": "Isa Gravelle",  "interests": [  ],  "children": [ {  "name": "Lashonda Gravelle" }, {  "name": "Carry Gravelle",  "age": 58 } ] }
+{  "cid": 5,  "name": "Heide Naifeh",  "interests": [ "Music", "Databases" ],  "children": [ {  "name": "Deirdre Naifeh" }, {  "name": "Jacquelyne Naifeh",  "age": 39 } ] }
+{  "cid": 531,  "name": "Camelia Yoes",  "interests": [  ],  "children": [  ] }
+{  "cid": 280,  "name": "Marlo Maung",  "interests": [ "Movies" ],  "children": [ {  "name": "Harold Maung" } ] }
+{  "cid": 507,  "name": "Yuk Flanegan",  "interests": [ "Puzzles", "Puzzles", "Squash" ],  "children": [ {  "name": "Alexander Flanegan" } ] }
+{  "cid": 865,  "name": "Moon Marino",  "age": 43,  "address": {  "number": 5710,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Skiing" ],  "children": [ {  "name": "Markita Marino",  "age": 10 } ] }
+{  "cid": 527,  "name": "Lance Kenison",  "age": 77,  "address": {  "number": 8750,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Squash", "Cooking", "Bass", "Puzzles" ],  "children": [ {  "name": "Youlanda Kenison" }, {  "name": "Lavon Kenison" }, {  "name": "Maryann Kenison",  "age": 60 }, {  "name": "Kecia Kenison",  "age": 50 } ] }
+{  "cid": 305,  "name": "Tuyet Leinbach",  "interests": [ "Puzzles", "Walking" ],  "children": [  ] }
+{  "cid": 75,  "name": "Monroe Fansher",  "interests": [ "Base Jumping", "Tennis", "Books", "Cigars" ],  "children": [ {  "name": "Honey Fansher" }, {  "name": "Sima Fansher",  "age": 22 }, {  "name": "Cassaundra Fansher" } ] }
+{  "cid": 320,  "name": "Charley Hermenegildo",  "interests": [  ],  "children": [ {  "name": "Melda Hermenegildo",  "age": 51 }, {  "name": "Lashon Hermenegildo" } ] }
+{  "cid": 13,  "name": "Nicol Kolmer",  "interests": [ "Coffee" ],  "children": [ {  "name": "Erika Kolmer",  "age": 40 }, {  "name": "Justin Kolmer" }, {  "name": "Dorathy Kolmer" }, {  "name": "Anastacia Kolmer",  "age": 27 } ] }
+{  "cid": 956,  "name": "Laquanda Bynoe",  "age": 79,  "address": {  "number": 6122,  "street": "Main St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Joel Bynoe" }, {  "name": "Brian Bynoe",  "age": 61 }, {  "name": "Shana Bynoe" } ] }
+{  "cid": 469,  "name": "Hilda Grabe",  "age": 36,  "address": {  "number": 9745,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Skiing", "Bass", "Coffee", "Music" ],  "children": [  ] }
+{  "cid": 558,  "name": "Dorie Schomer",  "age": 58,  "address": {  "number": 9295,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Fishing", "Bass", "Cigars", "Movies" ],  "children": [ {  "name": "Duncan Schomer" }, {  "name": "Donn Schomer",  "age": 14 }, {  "name": "Franklyn Schomer",  "age": 41 }, {  "name": "Valarie Schomer" } ] }
+{  "cid": 116,  "name": "Conrad Zozaya",  "age": 81,  "address": {  "number": 1667,  "street": "View St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Jenette Zozaya",  "age": 17 } ] }
+{  "cid": 870,  "name": "Natosha Lufsey",  "interests": [ "Cigars", "Walking" ],  "children": [ {  "name": "Tiffany Lufsey" } ] }
+{  "cid": 279,  "name": "Saundra Croan",  "interests": [ "Movies" ],  "children": [ {  "name": "Jena Croan",  "age": 37 }, {  "name": "Sarai Croan" }, {  "name": "Junita Croan" }, {  "name": "Ferdinand Croan",  "age": 43 } ] }
+{  "cid": 529,  "name": "Cinderella Lewis",  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Flor Lewis" }, {  "name": "Alonzo Lewis",  "age": 23 } ] }
+{  "cid": 749,  "name": "Pearle Mauney",  "interests": [  ],  "children": [ {  "name": "Delpha Mauney" }, {  "name": "Micki Mauney",  "age": 28 }, {  "name": "Wayne Mauney" } ] }
+{  "cid": 463,  "name": "Mika Rininger",  "interests": [ "Databases", "Cooking" ],  "children": [ {  "name": "Inez Rininger",  "age": 58 }, {  "name": "Betty Rininger" }, {  "name": "Laurie Rininger",  "age": 48 }, {  "name": "Billie Rininger" } ] }
+{  "cid": 329,  "name": "Dennis Cremins",  "interests": [ "Movies", "Fishing", "Music", "Squash" ],  "children": [ {  "name": "Destiny Cremins" }, {  "name": "Garret Cremins",  "age": 34 } ] }
+{  "cid": 247,  "name": "Minda Heron",  "age": 25,  "address": {  "number": 1629,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Tennis" ],  "children": [  ] }
+{  "cid": 729,  "name": "Karren Defrain",  "interests": [ "Books", "Walking", "Puzzles", "Tennis" ],  "children": [ {  "name": "Usha Defrain" }, {  "name": "Ahmed Defrain",  "age": 14 }, {  "name": "Kathryn Defrain" } ] }
+{  "cid": 872,  "name": "Michele Herschel",  "age": 39,  "address": {  "number": 4287,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [  ] }
+{  "cid": 758,  "name": "Akiko Hoenstine",  "age": 56,  "address": {  "number": 8888,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Movies", "Walking" ],  "children": [ {  "name": "Maren Hoenstine" }, {  "name": "Tyler Hoenstine" }, {  "name": "Jesse Hoenstine",  "age": 40 } ] }
+{  "cid": 685,  "name": "Lois Mcglothian",  "interests": [ "Movies", "Skiing" ],  "children": [ {  "name": "Karon Mcglothian",  "age": 35 } ] }
+{  "cid": 949,  "name": "Elissa Rogue",  "interests": [ "Fishing", "Music" ],  "children": [ {  "name": "Noriko Rogue",  "age": 41 }, {  "name": "Lavona Rogue",  "age": 39 } ] }
+{  "cid": 841,  "name": "Omar Enwall",  "interests": [ "Skiing", "Skiing", "Books" ],  "children": [ {  "name": "Kirby Enwall",  "age": 31 }, {  "name": "Cythia Enwall",  "age": 24 }, {  "name": "August Enwall" } ] }
+{  "cid": 936,  "name": "Berna Whyman",  "interests": [ "Bass", "Cooking", "Running", "Tennis" ],  "children": [ {  "name": "Marci Whyman",  "age": 10 }, {  "name": "Hyon Whyman" }, {  "name": "Jessia Whyman" } ] }
+{  "cid": 768,  "name": "Adelina Troendle",  "interests": [ "Computers" ],  "children": [ {  "name": "Lenna Troendle",  "age": 51 }, {  "name": "Ines Troendle",  "age": 48 }, {  "name": "Ora Troendle" } ] }
+{  "cid": 499,  "name": "Carlita Tarlton",  "age": 43,  "address": {  "number": 9148,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Computers", "Base Jumping", "Video Games" ],  "children": [  ] }
+{  "cid": 48,  "name": "Delia Salveson",  "age": 44,  "address": {  "number": 5596,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Cigars", "Running", "Walking", "Running" ],  "children": [ {  "name": "Logan Salveson",  "age": 21 }, {  "name": "Temple Salveson",  "age": 17 }, {  "name": "Kimi Salveson" }, {  "name": "Jacob Salveson",  "age": 20 } ] }
+{  "cid": 919,  "name": "Fairy Wansley",  "age": 45,  "address": {  "number": 9020,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Wine", "Walking", "Databases", "Video Games" ],  "children": [ {  "name": "Marvella Wansley" }, {  "name": "Hisako Wansley" }, {  "name": "Shaunta Wansley" }, {  "name": "Gemma Wansley",  "age": 21 } ] }
+{  "cid": 524,  "name": "Rickie Manche",  "interests": [  ],  "children": [  ] }
+{  "cid": 214,  "name": "Louvenia Zaffalon",  "interests": [ "Skiing", "Books" ],  "children": [  ] }
+{  "cid": 971,  "name": "Loura Paap",  "interests": [ "Walking", "Music", "Base Jumping", "Cooking" ],  "children": [ {  "name": "Eliza Paap",  "age": 54 }, {  "name": "Dortha Paap" }, {  "name": "Robin Paap" } ] }
+{  "cid": 125,  "name": "Leigh Pusey",  "interests": [  ],  "children": [ {  "name": "Elbert Pusey",  "age": 44 }, {  "name": "Golden Pusey" }, {  "name": "Maria Pusey" } ] }
+{  "cid": 733,  "name": "Edie Stager",  "age": 26,  "address": {  "number": 2691,  "street": "Park St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Ethyl Stager",  "age": 10 } ] }
+{  "cid": 644,  "name": "Julio Gilly",  "interests": [ "Puzzles" ],  "children": [ {  "name": "Eleonore Gilly" } ] }
+{  "cid": 693,  "name": "Ela Crisan",  "interests": [ "Movies" ],  "children": [  ] }
+{  "cid": 149,  "name": "Marcella Diamond",  "age": 62,  "address": {  "number": 720,  "street": "7th St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Ezra Diamond" } ] }
+{  "cid": 28,  "name": "Ariana Gillert",  "age": 54,  "address": {  "number": 7331,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Databases" ],  "children": [ {  "name": "Inge Gillert" }, {  "name": "Jeraldine Gillert",  "age": 13 } ] }
+{  "cid": 366,  "name": "Rosia Wenzinger",  "interests": [  ],  "children": [  ] }
+{  "cid": 501,  "name": "Alyce Coant",  "interests": [ "Music", "Base Jumping" ],  "children": [ {  "name": "Elyse Coant",  "age": 50 } ] }
+{  "cid": 705,  "name": "Sofia Bonniwell",  "age": 81,  "address": {  "number": 767,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Basketball" ],  "children": [ {  "name": "Douglass Bonniwell",  "age": 58 }, {  "name": "Jackeline Bonniwell",  "age": 16 } ] }
+{  "cid": 46,  "name": "Columbus Huntington",  "age": 22,  "address": {  "number": 3809,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Movies" ],  "children": [ {  "name": "Dana Huntington",  "age": 10 }, {  "name": "Rosa Huntington" } ] }
+{  "cid": 326,  "name": "Tad Tellers",  "interests": [ "Books", "Tennis", "Base Jumping" ],  "children": [ {  "name": "Fannie Tellers" } ] }
+{  "cid": 929,  "name": "Jean Guitierrez",  "age": 75,  "address": {  "number": 9736,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Wine", "Wine", "Fishing" ],  "children": [  ] }
+{  "cid": 227,  "name": "Carlos Skyes",  "interests": [  ],  "children": [ {  "name": "Cortney Skyes",  "age": 32 } ] }
+{  "cid": 965,  "name": "Mellie Risen",  "interests": [ "Tennis" ],  "children": [ {  "name": "Coreen Risen",  "age": 36 }, {  "name": "Faith Risen",  "age": 34 }, {  "name": "Crystle Risen",  "age": 54 } ] }
+{  "cid": 987,  "name": "Sharolyn Demchak",  "age": 36,  "address": {  "number": 4672,  "street": "Lake St.",  "city": "San Jose" },  "interests": [  ],  "children": [  ] }
+{  "cid": 972,  "name": "Ryan Dudgeon",  "interests": [ "Cigars", "Movies", "Cigars", "Books" ],  "children": [ {  "name": "Candelaria Dudgeon",  "age": 48 }, {  "name": "Donya Dudgeon" } ] }
+{  "cid": 716,  "name": "Deirdre Bruderer",  "interests": [ "Computers", "Wine" ],  "children": [ {  "name": "Coralee Bruderer" }, {  "name": "Mina Bruderer" }, {  "name": "Lindsey Bruderer",  "age": 35 }, {  "name": "Yi Bruderer" } ] }
+{  "cid": 83,  "name": "Filiberto Couillard",  "interests": [ "Cooking", "Books" ],  "children": [ {  "name": "Diane Couillard",  "age": 19 }, {  "name": "Asa Couillard",  "age": 23 }, {  "name": "Zaida Couillard",  "age": 57 }, {  "name": "Shavonne Couillard" } ] }
+{  "cid": 576,  "name": "Dean Waltenbaugh",  "age": 47,  "address": {  "number": 9478,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Music", "Base Jumping", "Puzzles", "Wine" ],  "children": [ {  "name": "Judy Waltenbaugh" }, {  "name": "Omer Waltenbaugh",  "age": 11 }, {  "name": "Samuel Waltenbaugh",  "age": 21 }, {  "name": "Neville Waltenbaugh" } ] }
+{  "cid": 372,  "name": "Zena Keglovic",  "age": 22,  "address": {  "number": 7675,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Basketball", "Wine" ],  "children": [  ] }
+{  "cid": 662,  "name": "Domonique Corbi",  "age": 13,  "address": {  "number": 7286,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Tennis", "Cooking", "Computers" ],  "children": [ {  "name": "Katrice Corbi" }, {  "name": "Idalia Corbi" }, {  "name": "Hayley Corbi" } ] }
+{  "cid": 983,  "name": "Leone Aucter",  "age": 48,  "address": {  "number": 4957,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Video Games", "Fishing", "Video Games", "Music" ],  "children": [ {  "name": "Clement Aucter",  "age": 32 }, {  "name": "Socorro Aucter",  "age": 35 } ] }
+{  "cid": 536,  "name": "Wilber Rehrer",  "interests": [ "Movies" ],  "children": [ {  "name": "Zulema Rehrer" }, {  "name": "Lavonda Rehrer" }, {  "name": "Stacey Rehrer",  "age": 59 } ] }
+{  "cid": 938,  "name": "Parthenia Dromgoole",  "age": 36,  "address": {  "number": 527,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Fishing" ],  "children": [  ] }
+{  "cid": 629,  "name": "Mayola Clabo",  "interests": [ "Basketball", "Skiing", "Running" ],  "children": [ {  "name": "Rigoberto Clabo",  "age": 58 } ] }
+{  "cid": 197,  "name": "Garth Giannitti",  "interests": [ "Coffee", "Cigars" ],  "children": [ {  "name": "Patsy Giannitti" }, {  "name": "Ray Giannitti",  "age": 35 }, {  "name": "Kamala Giannitti",  "age": 35 }, {  "name": "Lauran Giannitti",  "age": 25 } ] }
+{  "cid": 230,  "name": "Tobias Vicars",  "age": 66,  "address": {  "number": 638,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Wine", "Walking", "Books", "Walking" ],  "children": [  ] }
+{  "cid": 978,  "name": "Rudy Watsky",  "age": 32,  "address": {  "number": 2754,  "street": "Oak St.",  "city": "Seattle" },  "interests": [ "Cooking" ],  "children": [  ] }
+{  "cid": 478,  "name": "Sophia Whitt",  "age": 26,  "address": {  "number": 2787,  "street": "Park St.",  "city": "Mountain View" },  "interests": [ "Fishing", "Databases" ],  "children": [ {  "name": "Irving Whitt",  "age": 13 }, {  "name": "Jeannette Whitt" } ] }
+{  "cid": 727,  "name": "Valene Resecker",  "interests": [ "Music", "Wine", "Books", "Walking" ],  "children": [  ] }
+{  "cid": 511,  "name": "Sanda Franson",  "interests": [ "Music", "Cooking", "Books", "Cooking" ],  "children": [  ] }
+{  "cid": 557,  "name": "Kaitlyn Hilleman",  "age": 61,  "address": {  "number": 1076,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Corrie Hilleman",  "age": 31 }, {  "name": "Jovan Hilleman" }, {  "name": "Carmine Hilleman" } ] }
+{  "cid": 121,  "name": "Shiela Gaustad",  "interests": [  ],  "children": [ {  "name": "Phebe Gaustad" }, {  "name": "Mavis Gaustad" }, {  "name": "Zula Gaustad",  "age": 37 } ] }
+{  "cid": 316,  "name": "Patrina Whitting",  "age": 74,  "address": {  "number": 4772,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Music", "Video Games", "Bass" ],  "children": [ {  "name": "Rubye Whitting" } ] }
+{  "cid": 698,  "name": "Tawanna Zanin",  "age": 60,  "address": {  "number": 7979,  "street": "View St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Denny Zanin",  "age": 31 }, {  "name": "Danial Zanin",  "age": 43 }, {  "name": "Kenyetta Zanin" }, {  "name": "Aleisha Zanin" } ] }
+{  "cid": 272,  "name": "Frederick Valla",  "age": 15,  "address": {  "number": 6805,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Video Games" ],  "children": [ {  "name": "Carroll Valla" } ] }
+{  "cid": 440,  "name": "Rosie Shappen",  "interests": [ "Cooking", "Music", "Cigars" ],  "children": [ {  "name": "Jung Shappen",  "age": 11 } ] }
+{  "cid": 997,  "name": "Yesenia Gao",  "age": 38,  "address": {  "number": 5990,  "street": "View St.",  "city": "Portland" },  "interests": [ "Computers", "Computers", "Puzzles", "Puzzles" ],  "children": [ {  "name": "Jared Gao",  "age": 11 }, {  "name": "Sang Gao" }, {  "name": "Jeanne Gao",  "age": 13 }, {  "name": "Lavona Gao",  "age": 23 } ] }
+{  "cid": 352,  "name": "Bonny Sischo",  "interests": [ "Bass", "Movies", "Computers" ],  "children": [ {  "name": "Judith Sischo",  "age": 43 }, {  "name": "Adeline Sischo" }, {  "name": "Dayna Sischo" } ] }
+{  "cid": 818,  "name": "Nellie Whetzell",  "interests": [ "Walking" ],  "children": [  ] }
+{  "cid": 874,  "name": "Jamie Credille",  "age": 87,  "address": {  "number": 3351,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Walking", "Movies", "Bass", "Basketball" ],  "children": [ {  "name": "Shirly Credille" }, {  "name": "Digna Credille" }, {  "name": "Sabra Credille" }, {  "name": "Broderick Credille" } ] }
+{  "cid": 532,  "name": "Tania Fraklin",  "age": 38,  "address": {  "number": 2857,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Squash", "Databases" ],  "children": [  ] }
+{  "cid": 134,  "name": "Alica Frontiero",  "interests": [ "Puzzles" ],  "children": [  ] }
+{  "cid": 688,  "name": "Maryellen Leriche",  "interests": [ "Music", "Walking", "Skiing" ],  "children": [ {  "name": "Dorinda Leriche",  "age": 27 } ] }
+{  "cid": 168,  "name": "Carlotta Broderson",  "interests": [ "Skiing", "Video Games", "Squash", "Databases" ],  "children": [ {  "name": "Adolfo Broderson",  "age": 54 }, {  "name": "Vickie Broderson" } ] }
+{  "cid": 402,  "name": "Terrilyn Shinall",  "interests": [ "Computers", "Skiing", "Music" ],  "children": [ {  "name": "Minh Shinall" }, {  "name": "Diedre Shinall",  "age": 22 } ] }
+{  "cid": 984,  "name": "Janett Kitchens",  "age": 66,  "address": {  "number": 7558,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Coffee", "Movies", "Squash" ],  "children": [ {  "name": "Grayce Kitchens",  "age": 14 }, {  "name": "Dwayne Kitchens" }, {  "name": "Wilber Kitchens",  "age": 51 }, {  "name": "Nancey Kitchens" } ] }
+{  "cid": 222,  "name": "Malcom Bloomgren",  "age": 39,  "address": {  "number": 4674,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Databases", "Skiing" ],  "children": [ {  "name": "Rosia Bloomgren" }, {  "name": "Bryant Bloomgren",  "age": 15 }, {  "name": "Donnie Bloomgren" } ] }
+{  "cid": 809,  "name": "Dagny Mangiaracina",  "age": 44,  "address": {  "number": 5993,  "street": "Lake St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Bari Mangiaracina",  "age": 31 }, {  "name": "Tiara Mangiaracina",  "age": 12 }, {  "name": "Milly Mangiaracina" }, {  "name": "Chelsie Mangiaracina" } ] }
+{  "cid": 769,  "name": "Isaias Tenny",  "age": 71,  "address": {  "number": 270,  "street": "Park St.",  "city": "Portland" },  "interests": [ "Wine", "Fishing", "Base Jumping" ],  "children": [ {  "name": "Theo Tenny" }, {  "name": "Shena Tenny" }, {  "name": "Coralee Tenny" }, {  "name": "Orval Tenny",  "age": 39 } ] }
+{  "cid": 159,  "name": "Jeanmarie Franchini",  "interests": [ "Music" ],  "children": [ {  "name": "Nikita Franchini" }, {  "name": "Willetta Franchini" }, {  "name": "Ester Franchini",  "age": 12 } ] }
+{  "cid": 198,  "name": "Thelma Youkers",  "interests": [ "Basketball", "Movies", "Cooking" ],  "children": [ {  "name": "Shamika Youkers",  "age": 28 } ] }
+{  "cid": 856,  "name": "Inocencia Petzold",  "age": 83,  "address": {  "number": 4631,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Basketball", "Squash", "Movies", "Base Jumping" ],  "children": [  ] }
+{  "cid": 394,  "name": "Lizette Roux",  "age": 57,  "address": {  "number": 458,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Bass", "Books" ],  "children": [ {  "name": "Doloris Roux" } ] }
+{  "cid": 560,  "name": "Karin Dicesare",  "interests": [ "Wine", "Puzzles" ],  "children": [  ] }
+{  "cid": 813,  "name": "Leann Domagala",  "age": 47,  "address": {  "number": 4472,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Computers" ],  "children": [ {  "name": "Alvera Domagala",  "age": 36 }, {  "name": "Rosalva Domagala",  "age": 27 }, {  "name": "Eugenia Domagala" }, {  "name": "My Domagala",  "age": 32 } ] }
+{  "cid": 165,  "name": "Melodie Starrick",  "interests": [ "Walking" ],  "children": [ {  "name": "Adria Starrick" }, {  "name": "Tasha Starrick",  "age": 25 } ] }
+{  "cid": 268,  "name": "Fernando Pingel",  "interests": [ "Computers", "Tennis", "Books" ],  "children": [ {  "name": "Latrice Pingel" }, {  "name": "Wade Pingel",  "age": 13 }, {  "name": "Christal Pingel" }, {  "name": "Melania Pingel" } ] }
+{  "cid": 408,  "name": "Ava Zornes",  "interests": [ "Music" ],  "children": [  ] }
+{  "cid": 258,  "name": "Florentina Hense",  "age": 20,  "address": {  "number": 8495,  "street": "View St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Noelle Hense" }, {  "name": "Roxann Hense" } ] }
+{  "cid": 59,  "name": "Rea Villicana",  "interests": [  ],  "children": [  ] }
+{  "cid": 774,  "name": "Nadene Rigel",  "interests": [ "Cigars", "Cigars" ],  "children": [ {  "name": "Rebbeca Rigel",  "age": 33 } ] }
+{  "cid": 606,  "name": "Virgilio Liebelt",  "age": 11,  "address": {  "number": 8348,  "street": "Cedar St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Stanford Liebelt" }, {  "name": "Delaine Liebelt" }, {  "name": "Kevin Liebelt" }, {  "name": "Michaele Liebelt" } ] }
+{  "cid": 848,  "name": "Myrta Kopf",  "interests": [ "Wine", "Basketball", "Base Jumping" ],  "children": [  ] }
+{  "cid": 817,  "name": "Missy Perdue",  "age": 59,  "address": {  "number": 2876,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [ "Basketball", "Cigars", "Computers", "Books" ],  "children": [ {  "name": "Shellie Perdue" }, {  "name": "Marx Perdue" }, {  "name": "Peg Perdue",  "age": 39 }, {  "name": "Dalton Perdue",  "age": 32 } ] }
+{  "cid": 615,  "name": "Kimber Warnberg",  "age": 77,  "address": {  "number": 1404,  "street": "View St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Kristal Warnberg" } ] }
+{  "cid": 623,  "name": "Lorna Krason",  "age": 40,  "address": {  "number": 9398,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Cigars", "Cigars", "Video Games", "Wine" ],  "children": [  ] }
+{  "cid": 90,  "name": "Dorethea Korns",  "interests": [ "Cooking", "Computers" ],  "children": [ {  "name": "Catheryn Korns",  "age": 22 } ] }
+{  "cid": 467,  "name": "Magali Ingerson",  "interests": [ "Books", "Base Jumping" ],  "children": [ {  "name": "Monty Ingerson",  "age": 11 }, {  "name": "Noelia Ingerson",  "age": 47 }, {  "name": "Tennie Ingerson" }, {  "name": "Merrill Ingerson" } ] }
+{  "cid": 11,  "name": "Meta Simek",  "age": 13,  "address": {  "number": 4384,  "street": "7th St.",  "city": "San Jose" },  "interests": [ "Wine", "Walking" ],  "children": [ {  "name": "Oretha Simek" }, {  "name": "Terence Simek" } ] }
+{  "cid": 175,  "name": "Loise Obhof",  "interests": [  ],  "children": [ {  "name": "Susann Obhof" }, {  "name": "Signe Obhof",  "age": 38 } ] }
+{  "cid": 665,  "name": "Garnet Desai",  "interests": [ "Databases" ],  "children": [ {  "name": "Aliza Desai" } ] }
+{  "cid": 588,  "name": "Debora Laughinghouse",  "age": 87,  "address": {  "number": 5099,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Tennis", "Walking", "Databases" ],  "children": [ {  "name": "Frederica Laughinghouse",  "age": 59 }, {  "name": "Johnie Laughinghouse",  "age": 12 }, {  "name": "Numbers Laughinghouse",  "age": 73 } ] }
+{  "cid": 66,  "name": "Lenny Latson",  "interests": [ "Music", "Video Games" ],  "children": [  ] }
+{  "cid": 745,  "name": "Tabatha Hagwell",  "interests": [  ],  "children": [ {  "name": "Gaynell Hagwell" } ] }
+{  "cid": 726,  "name": "Brinda Raudebaugh",  "age": 83,  "address": {  "number": 7179,  "street": "View St.",  "city": "Mountain View" },  "interests": [  ],  "children": [  ] }
+{  "cid": 371,  "name": "Agatha Tensley",  "age": 13,  "address": {  "number": 1810,  "street": "Hill St.",  "city": "San Jose" },  "interests": [ "Bass", "Running", "Movies" ],  "children": [ {  "name": "Launa Tensley" } ] }
+{  "cid": 474,  "name": "Claudie Hunstad",  "age": 46,  "address": {  "number": 3347,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Music", "Base Jumping", "Computers", "Cooking" ],  "children": [ {  "name": "Elanor Hunstad",  "age": 35 } ] }
+{  "cid": 761,  "name": "Adele Henrikson",  "interests": [ "Cooking", "Bass" ],  "children": [ {  "name": "Paulina Henrikson" }, {  "name": "David Henrikson" }, {  "name": "Jose Henrikson" }, {  "name": "Meg Henrikson" } ] }
+{  "cid": 442,  "name": "Val Disorda",  "interests": [ "Bass" ],  "children": [ {  "name": "Simone Disorda",  "age": 53 }, {  "name": "Jacalyn Disorda",  "age": 41 }, {  "name": "Ron Disorda" }, {  "name": "Clifton Disorda" } ] }
+{  "cid": 892,  "name": "Madge Hendson",  "age": 79,  "address": {  "number": 8832,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Databases", "Fishing", "Skiing" ],  "children": [ {  "name": "Elia Hendson",  "age": 48 }, {  "name": "Lashawn Hendson",  "age": 27 } ] }
+{  "cid": 244,  "name": "Rene Shenk",  "interests": [ "Puzzles", "Puzzles", "Skiing" ],  "children": [ {  "name": "Victor Shenk",  "age": 28 }, {  "name": "Doris Shenk" }, {  "name": "Max Shenk",  "age": 51 } ] }
+{  "cid": 375,  "name": "Chia Sagaser",  "age": 15,  "address": {  "number": 6025,  "street": "Park St.",  "city": "Mountain View" },  "interests": [ "Skiing" ],  "children": [ {  "name": "Garnet Sagaser" }, {  "name": "Mario Sagaser" }, {  "name": "Sun Sagaser" } ] }
+{  "cid": 344,  "name": "Aleshia Hongeva",  "age": 70,  "address": {  "number": 4092,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Books", "Video Games", "Puzzles", "Music" ],  "children": [  ] }
+{  "cid": 127,  "name": "Christian Anthes",  "age": 32,  "address": {  "number": 6258,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Running", "Bass" ],  "children": [ {  "name": "Sophia Anthes" } ] }
+{  "cid": 182,  "name": "Christiana Westlie",  "interests": [ "Skiing", "Bass" ],  "children": [ {  "name": "Ilda Westlie",  "age": 18 } ] }
+{  "cid": 787,  "name": "Sara Yerly",  "age": 12,  "address": {  "number": 872,  "street": "7th St.",  "city": "Seattle" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Nettie Yerly" }, {  "name": "Regine Yerly" }, {  "name": "Hyo Yerly" } ] }
+{  "cid": 405,  "name": "Shawnda Landborg",  "age": 73,  "address": {  "number": 2396,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Cherrie Landborg",  "age": 10 } ] }
+{  "cid": 530,  "name": "Olevia Sturk",  "age": 72,  "address": {  "number": 1939,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [ "Computers" ],  "children": [ {  "name": "Cindy Sturk",  "age": 18 }, {  "name": "Alishia Sturk" }, {  "name": "Sonja Sturk",  "age": 51 } ] }
+{  "cid": 152,  "name": "Karyn Cockburn",  "interests": [ "Puzzles", "Cigars", "Bass", "Computers" ],  "children": [ {  "name": "Zenobia Cockburn",  "age": 44 }, {  "name": "Shellie Cockburn" }, {  "name": "Kermit Cockburn" } ] }
+{  "cid": 164,  "name": "Lucrecia Dahlhauser",  "interests": [ "Wine" ],  "children": [  ] }
+{  "cid": 322,  "name": "Jaclyn Ettl",  "age": 83,  "address": {  "number": 4500,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Databases", "Skiing" ],  "children": [ {  "name": "Noah Ettl",  "age": 30 }, {  "name": "Kesha Ettl" } ] }
+{  "cid": 556,  "name": "Dalene Mateen",  "age": 76,  "address": {  "number": 2854,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Video Games", "Walking", "Databases", "Cooking" ],  "children": [ {  "name": "Jazmin Mateen",  "age": 29 } ] }
+{  "cid": 105,  "name": "Camilla Lohman",  "interests": [  ],  "children": [ {  "name": "Melania Lohman",  "age": 50 }, {  "name": "Mike Lohman",  "age": 53 }, {  "name": "Cassaundra Lohman",  "age": 32 }, {  "name": "Jay Lohman" } ] }
+{  "cid": 26,  "name": "Jone Okuna",  "age": 78,  "address": {  "number": 6006,  "street": "7th St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Franchesca Okuna" }, {  "name": "Fred Okuna",  "age": 17 }, {  "name": "Marcellus Okuna" } ] }
+{  "cid": 823,  "name": "Deloras Scorzelli",  "age": 54,  "address": {  "number": 6140,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Bass", "Fishing", "Databases", "Fishing" ],  "children": [ {  "name": "Catharine Scorzelli",  "age": 12 }, {  "name": "Margarite Scorzelli",  "age": 19 }, {  "name": "Neomi Scorzelli",  "age": 38 }, {  "name": "Ossie Scorzelli" } ] }
+{  "cid": 12,  "name": "Laurinda Raimann",  "interests": [ "Basketball", "Coffee" ],  "children": [ {  "name": "Lulu Raimann" }, {  "name": "Refugia Raimann",  "age": 19 }, {  "name": "Jimmie Raimann",  "age": 10 }, {  "name": "Cindy Raimann" } ] }
+{  "cid": 832,  "name": "Alina Hosley",  "interests": [ "Databases", "Databases", "Music" ],  "children": [ {  "name": "Sebrina Hosley" }, {  "name": "Dyan Hosley" } ] }
+{  "cid": 104,  "name": "Neda Dilts",  "interests": [ "Basketball" ],  "children": [ {  "name": "Nona Dilts",  "age": 28 }, {  "name": "Wm Dilts" }, {  "name": "Svetlana Dilts",  "age": 46 }, {  "name": "Iva Dilts",  "age": 59 } ] }
+{  "cid": 836,  "name": "Elden Shumski",  "interests": [  ],  "children": [ {  "name": "Weldon Shumski" }, {  "name": "Anneliese Shumski" } ] }
+{  "cid": 265,  "name": "Donte Stempien",  "age": 25,  "address": {  "number": 3882,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Wine", "Books" ],  "children": [  ] }
+{  "cid": 578,  "name": "Dolly Delphia",  "interests": [ "Wine" ],  "children": [ {  "name": "Sharron Delphia" }, {  "name": "Shemeka Delphia" }, {  "name": "Rachael Delphia" } ] }
+{  "cid": 954,  "name": "Yolonda Pu",  "interests": [ "Video Games", "Music", "Cooking", "Skiing" ],  "children": [ {  "name": "Josephina Pu",  "age": 35 } ] }
+{  "cid": 722,  "name": "Noel Goncalves",  "interests": [ "Books", "Bass", "Books", "Books" ],  "children": [ {  "name": "Latrice Goncalves" }, {  "name": "Evelia Goncalves",  "age": 36 }, {  "name": "Etta Goncalves",  "age": 11 }, {  "name": "Collin Goncalves" } ] }
+{  "cid": 224,  "name": "Rene Rowey",  "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ],  "children": [ {  "name": "Necole Rowey",  "age": 26 }, {  "name": "Sharyl Rowey",  "age": 20 }, {  "name": "Yvone Rowey",  "age": 36 } ] }
+{  "cid": 739,  "name": "Libbie Thigpin",  "interests": [ "Databases" ],  "children": [  ] }
+{  "cid": 641,  "name": "Barney Perz",  "interests": [ "Running", "Running", "Databases", "Running" ],  "children": [ {  "name": "Cristie Perz" }, {  "name": "Troy Perz",  "age": 38 } ] }
+{  "cid": 212,  "name": "Christi Vichi",  "interests": [ "Squash" ],  "children": [  ] }
+{  "cid": 337,  "name": "Kay Durney",  "age": 52,  "address": {  "number": 4203,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Walking" ],  "children": [ {  "name": "Velia Durney",  "age": 38 }, {  "name": "Erin Durney" } ] }
+{  "cid": 979,  "name": "Yoko Bailony",  "interests": [  ],  "children": [ {  "name": "Vivienne Bailony" }, {  "name": "Lori Bailony",  "age": 47 } ] }
+{  "cid": 312,  "name": "Epifania Chorney",  "age": 62,  "address": {  "number": 9749,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Wine", "Puzzles", "Tennis" ],  "children": [ {  "name": "Lizeth Chorney",  "age": 22 } ] }
+{  "cid": 603,  "name": "Barry Corkum",  "interests": [ "Running", "Running" ],  "children": [ {  "name": "Charlesetta Corkum" }, {  "name": "Helaine Corkum" }, {  "name": "Erinn Corkum",  "age": 28 }, {  "name": "Alesia Corkum",  "age": 36 } ] }
+{  "cid": 554,  "name": "Darci Yafai",  "age": 60,  "address": {  "number": 4694,  "street": "Park St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Lecia Yafai",  "age": 47 } ] }
+{  "cid": 678,  "name": "Lekisha Barnell",  "interests": [ "Movies", "Skiing", "Running" ],  "children": [ {  "name": "August Barnell" }, {  "name": "Tiffany Barnell",  "age": 55 }, {  "name": "Meghan Barnell" } ] }
+{  "cid": 582,  "name": "Suzie Ocallahan",  "interests": [ "Basketball" ],  "children": [ {  "name": "Tamra Ocallahan" } ] }
+{  "cid": 193,  "name": "Melisa Maccarter",  "age": 50,  "address": {  "number": 1494,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Basketball" ],  "children": [ {  "name": "Yetta Maccarter" }, {  "name": "Geralyn Maccarter" } ] }
+{  "cid": 430,  "name": "Cari Woll",  "age": 45,  "address": {  "number": 8226,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Cooking", "Walking", "Cooking" ],  "children": [ {  "name": "Tomasa Woll",  "age": 32 }, {  "name": "Annika Woll",  "age": 21 } ] }
+{  "cid": 118,  "name": "Ellis Skillom",  "age": 78,  "address": {  "number": 9337,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Running", "Cigars" ],  "children": [ {  "name": "Emory Skillom" } ] }
+{  "cid": 522,  "name": "Daryl Kissack",  "age": 86,  "address": {  "number": 7825,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Squash", "Base Jumping", "Tennis" ],  "children": [ {  "name": "Darrel Kissack",  "age": 21 } ] }
+{  "cid": 151,  "name": "Charlyn Soyars",  "age": 21,  "address": {  "number": 2796,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [  ] }
+{  "cid": 510,  "name": "Candace Morello",  "interests": [ "Wine", "Base Jumping", "Running" ],  "children": [ {  "name": "Sandy Morello",  "age": 57 }, {  "name": "Delois Morello",  "age": 15 } ] }
+{  "cid": 655,  "name": "Shaun Brandenburg",  "interests": [ "Skiing", "Computers", "Base Jumping" ],  "children": [ {  "name": "Ned Brandenburg" }, {  "name": "Takako Brandenburg",  "age": 41 }, {  "name": "Astrid Brandenburg" }, {  "name": "Patience Brandenburg" } ] }
+{  "cid": 308,  "name": "Solomon Schwenke",  "interests": [ "Puzzles" ],  "children": [ {  "name": "Gertrude Schwenke" }, {  "name": "Marcell Schwenke",  "age": 41 }, {  "name": "Shalon Schwenke" } ] }
+{  "cid": 502,  "name": "Lawana Mulik",  "age": 82,  "address": {  "number": 3071,  "street": "Park St.",  "city": "Portland" },  "interests": [ "Cigars", "Cigars" ],  "children": [ {  "name": "Carrie Mulik" }, {  "name": "Sharlene Mulik",  "age": 33 }, {  "name": "Leone Mulik",  "age": 46 } ] }
+{  "cid": 740,  "name": "Thomasine Collado",  "interests": [ "Music" ],  "children": [ {  "name": "Tabetha Collado" }, {  "name": "Alline Collado" }, {  "name": "Delisa Collado" }, {  "name": "Jack Collado",  "age": 56 } ] }
+{  "cid": 386,  "name": "Mao Gradowski",  "age": 36,  "address": {  "number": 5116,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Computers", "Fishing" ],  "children": [ {  "name": "Jeneva Gradowski" }, {  "name": "Thu Gradowski",  "age": 22 }, {  "name": "Daphine Gradowski" }, {  "name": "Providencia Gradowski" } ] }
+{  "cid": 783,  "name": "Johnnie Kesby",  "age": 56,  "address": {  "number": 9798,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Puzzles", "Tennis" ],  "children": [  ] }
+{  "cid": 694,  "name": "Ariel Soltani",  "interests": [ "Databases", "Music", "Puzzles" ],  "children": [ {  "name": "Aldo Soltani" }, {  "name": "Anglea Soltani" } ] }
+{  "cid": 128,  "name": "Edwin Harwick",  "interests": [ "Fishing", "Squash", "Basketball" ],  "children": [ {  "name": "Tomeka Harwick",  "age": 34 }, {  "name": "Caroline Harwick",  "age": 57 }, {  "name": "Peter Harwick" }, {  "name": "Adele Harwick" } ] }
+{  "cid": 348,  "name": "Matthew Pantaleo",  "age": 80,  "address": {  "number": 9782,  "street": "Washington St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Faviola Pantaleo" }, {  "name": "Yang Pantaleo" }, {  "name": "Christopher Pantaleo" }, {  "name": "Jacqui Pantaleo",  "age": 58 } ] }
+{  "cid": 22,  "name": "Sarita Burrer",  "interests": [ "Cigars", "Computers" ],  "children": [  ] }
+{  "cid": 44,  "name": "Agustin Clubs",  "interests": [  ],  "children": [ {  "name": "Maxwell Clubs",  "age": 31 }, {  "name": "Rayna Clubs" }, {  "name": "Darwin Clubs" } ] }
+{  "cid": 327,  "name": "Minnie Scali",  "interests": [ "Cooking", "Squash", "Skiing" ],  "children": [ {  "name": "Jalisa Scali" }, {  "name": "Preston Scali" }, {  "name": "Stephani Scali",  "age": 47 }, {  "name": "Candra Scali" } ] }
+{  "cid": 668,  "name": "Dorene Spigelman",  "interests": [  ],  "children": [ {  "name": "Chiquita Spigelman",  "age": 29 }, {  "name": "Anisha Spigelman",  "age": 34 }, {  "name": "Micah Spigelman",  "age": 28 } ] }
+{  "cid": 53,  "name": "Ricardo Greiwe",  "age": 24,  "address": {  "number": 8983,  "street": "View St.",  "city": "Portland" },  "interests": [  ],  "children": [  ] }
+{  "cid": 157,  "name": "Mckenzie Tahir",  "age": 78,  "address": {  "number": 6752,  "street": "Hill St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Margarita Tahir",  "age": 18 }, {  "name": "Mia Tahir",  "age": 47 }, {  "name": "Gaylord Tahir" } ] }
+{  "cid": 504,  "name": "Marla Kolenda",  "age": 57,  "address": {  "number": 464,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Coffee" ],  "children": [ {  "name": "Iliana Kolenda",  "age": 34 }, {  "name": "Ammie Kolenda",  "age": 20 }, {  "name": "Candi Kolenda",  "age": 23 }, {  "name": "Lyla Kolenda",  "age": 23 } ] }
+{  "cid": 601,  "name": "Zackary Willier",  "interests": [ "Cooking", "Databases", "Databases" ],  "children": [  ] }
+{  "cid": 837,  "name": "Denice Wolken",  "age": 28,  "address": {  "number": 5010,  "street": "7th St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Kattie Wolken" } ] }
+{  "cid": 144,  "name": "Celesta Sosebee",  "age": 19,  "address": {  "number": 2683,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Databases", "Databases" ],  "children": [ {  "name": "Jesse Sosebee" }, {  "name": "Oralee Sosebee" }, {  "name": "Sunday Sosebee" } ] }
+{  "cid": 651,  "name": "Delana Henk",  "age": 69,  "address": {  "number": 5497,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Coffee", "Video Games", "Databases" ],  "children": [ {  "name": "Loan Henk" }, {  "name": "Teresa Henk",  "age": 20 }, {  "name": "Randell Henk" }, {  "name": "Micah Henk" } ] }
+{  "cid": 897,  "name": "Gerald Roehrman",  "interests": [ "Bass", "Wine" ],  "children": [ {  "name": "Virgie Roehrman",  "age": 28 }, {  "name": "Akiko Roehrman",  "age": 59 }, {  "name": "Robbie Roehrman",  "age": 10 }, {  "name": "Flavia Roehrman" } ] }
+{  "cid": 671,  "name": "Harley Emami",  "interests": [ "Basketball" ],  "children": [ {  "name": "Valentine Emami" }, {  "name": "Pearlene Emami" } ] }
+{  "cid": 406,  "name": "Addie Mandez",  "interests": [ "Tennis", "Cigars", "Books" ],  "children": [ {  "name": "Rosendo Mandez",  "age": 34 } ] }
+{  "cid": 955,  "name": "Liliana Stenkamp",  "interests": [ "Music" ],  "children": [  ] }
+{  "cid": 519,  "name": "Julianna Goodsell",  "age": 59,  "address": {  "number": 5594,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Video Games", "Fishing" ],  "children": [  ] }
+{  "cid": 714,  "name": "Felipe Gobel",  "interests": [ "Coffee", "Cigars", "Cooking", "Squash" ],  "children": [ {  "name": "Hortense Gobel",  "age": 15 }, {  "name": "Thomas Gobel",  "age": 25 }, {  "name": "Deena Gobel",  "age": 53 }, {  "name": "Shelby Gobel" } ] }
+{  "cid": 717,  "name": "Paulette Moccasin",  "age": 87,  "address": {  "number": 1426,  "street": "View St.",  "city": "Portland" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Savannah Moccasin" }, {  "name": "Mariela Moccasin",  "age": 34 }, {  "name": "Isadora Moccasin" }, {  "name": "Vivien Moccasin",  "age": 31 } ] }
+{  "cid": 982,  "name": "Jude Brandsrud",  "age": 41,  "address": {  "number": 7133,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Bass", "Skiing" ],  "children": [ {  "name": "Scottie Brandsrud" }, {  "name": "Gennie Brandsrud",  "age": 10 }, {  "name": "Agnes Brandsrud" }, {  "name": "Clarinda Brandsrud",  "age": 17 } ] }
+{  "cid": 254,  "name": "Jeanice Longanecker",  "age": 74,  "address": {  "number": 2613,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Books", "Base Jumping" ],  "children": [  ] }
+{  "cid": 890,  "name": "Janise Maccarthy",  "age": 66,  "address": {  "number": 7337,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Wine", "Computers" ],  "children": [  ] }
+{  "cid": 996,  "name": "Elouise Wider",  "interests": [ "Coffee", "Computers", "Base Jumping" ],  "children": [  ] }
+{  "cid": 245,  "name": "Lupe Abshear",  "age": 55,  "address": {  "number": 7269,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Song Abshear" }, {  "name": "Honey Abshear",  "age": 31 } ] }
+{  "cid": 645,  "name": "Shawnda Dollinger",  "age": 36,  "address": {  "number": 5980,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Vicente Dollinger" }, {  "name": "Kerrie Dollinger",  "age": 10 }, {  "name": "Sima Dollinger",  "age": 14 } ] }
+{  "cid": 448,  "name": "Gracie Pekas",  "age": 59,  "address": {  "number": 4732,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Base Jumping", "Wine", "Cigars" ],  "children": [ {  "name": "Jeanett Pekas",  "age": 35 }, {  "name": "Jennifer Pekas" }, {  "name": "Carrol Pekas" } ] }
+{  "cid": 123,  "name": "Marian Courrege",  "age": 30,  "address": {  "number": 7321,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Coffee" ],  "children": [  ] }
+{  "cid": 732,  "name": "Dania Fabio",  "interests": [ "Skiing" ],  "children": [ {  "name": "Virgie Fabio" }, {  "name": "Nereida Fabio",  "age": 37 } ] }
+{  "cid": 589,  "name": "Rebeca Blackwell",  "age": 66,  "address": {  "number": 5708,  "street": "View St.",  "city": "Portland" },  "interests": [  ],  "children": [  ] }
+{  "cid": 731,  "name": "Yajaira Orto",  "interests": [ "Music", "Databases" ],  "children": [ {  "name": "Eliz Orto",  "age": 17 }, {  "name": "Gisela Orto" } ] }
+{  "cid": 797,  "name": "Frederica Kale",  "age": 77,  "address": {  "number": 6861,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Puzzles", "Bass" ],  "children": [ {  "name": "Shanice Kale" }, {  "name": "Soraya Kale",  "age": 64 }, {  "name": "Laurena Kale",  "age": 57 } ] }
+{  "cid": 1,  "name": "Trudie Minick",  "age": 75,  "address": {  "number": 6740,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Fishing", "Squash" ],  "children": [ {  "name": "Arie Minick",  "age": 56 }, {  "name": "Alline Minick",  "age": 57 }, {  "name": "Petronila Minick",  "age": 56 } ] }
+{  "cid": 289,  "name": "Clarence Milette",  "age": 16,  "address": {  "number": 3778,  "street": "Oak St.",  "city": "Seattle" },  "interests": [ "Books", "Base Jumping", "Music" ],  "children": [  ] }
+{  "cid": 846,  "name": "Kieth Norlund",  "age": 15,  "address": {  "number": 4039,  "street": "Park St.",  "city": "Mountain View" },  "interests": [ "Wine", "Walking", "Puzzles" ],  "children": [ {  "name": "Shawn Norlund" } ] }
+{  "cid": 437,  "name": "Marlene Macintyre",  "age": 86,  "address": {  "number": 3708,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Wine", "Walking", "Music", "Coffee" ],  "children": [ {  "name": "Todd Macintyre" }, {  "name": "Mechelle Macintyre",  "age": 50 } ] }
+{  "cid": 881,  "name": "Leora Chesnutt",  "age": 49,  "address": {  "number": 6487,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Movies" ],  "children": [ {  "name": "Myrtle Chesnutt" }, {  "name": "Serina Chesnutt",  "age": 11 }, {  "name": "Jana Chesnutt",  "age": 10 } ] }
+{  "cid": 219,  "name": "Joelle Valazquez",  "age": 73,  "address": {  "number": 9775,  "street": "Park St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Gene Valazquez" }, {  "name": "Ilona Valazquez" } ] }
+{  "cid": 191,  "name": "Lula Pangburn",  "age": 42,  "address": {  "number": 1309,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Skiing", "Cooking", "Walking", "Video Games" ],  "children": [ {  "name": "Love Pangburn",  "age": 11 }, {  "name": "Bryant Pangburn",  "age": 13 }, {  "name": "Kenda Pangburn",  "age": 14 } ] }
+{  "cid": 184,  "name": "Mirtha Ricciardi",  "interests": [ "Music" ],  "children": [ {  "name": "Elsa Ricciardi",  "age": 30 }, {  "name": "Vicente Ricciardi" }, {  "name": "Sau Ricciardi",  "age": 28 } ] }
+{  "cid": 539,  "name": "Nicky Graceffo",  "interests": [ "Video Games" ],  "children": [  ] }
+{  "cid": 49,  "name": "Asa Schwing",  "age": 70,  "address": {  "number": 2261,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Tennis" ],  "children": [ {  "name": "Joy Schwing",  "age": 15 } ] }
+{  "cid": 92,  "name": "Kenny Laychock",  "age": 15,  "address": {  "number": 4790,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Video Games", "Basketball" ],  "children": [  ] }
+{  "cid": 19,  "name": "Nolan Yaish",  "age": 26,  "address": {  "number": 571,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Fishing", "Running", "Tennis", "Running" ],  "children": [ {  "name": "Jerold Yaish" }, {  "name": "Leatrice Yaish" }, {  "name": "Cletus Yaish",  "age": 10 } ] }
+{  "cid": 791,  "name": "Jame Apresa",  "age": 66,  "address": {  "number": 8417,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Running", "Puzzles", "Base Jumping" ],  "children": [ {  "name": "Awilda Apresa" }, {  "name": "Nelle Apresa",  "age": 40 }, {  "name": "Terrell Apresa" }, {  "name": "Malia Apresa",  "age": 43 } ] }
+{  "cid": 95,  "name": "Gavin Locey",  "age": 86,  "address": {  "number": 8162,  "street": "Lake St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Terrell Locey" }, {  "name": "Kazuko Locey",  "age": 36 }, {  "name": "Risa Locey" }, {  "name": "Dorethea Locey",  "age": 13 } ] }
+{  "cid": 790,  "name": "Dustin Brumble",  "interests": [ "Computers", "Databases", "Tennis" ],  "children": [ {  "name": "Oda Brumble" }, {  "name": "Jennefer Brumble",  "age": 26 }, {  "name": "Ricardo Brumble",  "age": 37 }, {  "name": "Graciela Brumble",  "age": 10 } ] }
+{  "cid": 824,  "name": "Vonda Czaplewski",  "age": 72,  "address": {  "number": 4597,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Skiing" ],  "children": [ {  "name": "Gaynelle Czaplewski" }, {  "name": "India Czaplewski" } ] }
+{  "cid": 281,  "name": "Ivey Riveria",  "interests": [ "Cooking", "Puzzles", "Fishing", "Wine" ],  "children": [ {  "name": "Mohamed Riveria" }, {  "name": "Dia Riveria",  "age": 17 }, {  "name": "Hope Riveria" } ] }
+{  "cid": 624,  "name": "Bong Lyall",  "interests": [ "Databases", "Music", "Video Games" ],  "children": [  ] }
+{  "cid": 754,  "name": "Luetta Joern",  "age": 25,  "address": {  "number": 5554,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Hildegarde Joern" }, {  "name": "Lorenza Joern",  "age": 13 } ] }
+{  "cid": 466,  "name": "Paulene Bagen",  "age": 87,  "address": {  "number": 4093,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Music" ],  "children": [ {  "name": "Antione Bagen" }, {  "name": "Samatha Bagen" } ] }
+{  "cid": 194,  "name": "Leslee Apking",  "age": 41,  "address": {  "number": 8107,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Irena Apking" }, {  "name": "Arla Apking" } ] }
+{  "cid": 620,  "name": "Arielle Mackellar",  "interests": [ "Cooking", "Bass" ],  "children": [ {  "name": "Evelin Mackellar",  "age": 17 }, {  "name": "Theresa Mackellar",  "age": 53 }, {  "name": "Ronnie Mackellar" }, {  "name": "Elwanda Mackellar",  "age": 54 } ] }
+{  "cid": 351,  "name": "Samual Alsandor",  "age": 68,  "address": {  "number": 33,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Bass", "Cigars", "Cooking", "Coffee" ],  "children": [  ] }
+{  "cid": 517,  "name": "Alfonso Bruderer",  "interests": [ "Bass" ],  "children": [  ] }
+{  "cid": 913,  "name": "Evelynn Fague",  "age": 42,  "address": {  "number": 5729,  "street": "7th St.",  "city": "Seattle" },  "interests": [ "Books", "Databases", "Cooking" ],  "children": [  ] }
+{  "cid": 541,  "name": "Sammy Adamitis",  "age": 71,  "address": {  "number": 5593,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Books", "Tennis", "Cooking" ],  "children": [  ] }
+{  "cid": 298,  "name": "Brittny Christin",  "interests": [ "Databases", "Video Games" ],  "children": [ {  "name": "Hilario Christin" }, {  "name": "Clarine Christin" } ] }
+{  "cid": 489,  "name": "Brigid Delosier",  "age": 31,  "address": {  "number": 6082,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Tennis", "Cigars", "Music" ],  "children": [ {  "name": "Allegra Delosier" }, {  "name": "Yong Delosier",  "age": 10 }, {  "name": "Steffanie Delosier",  "age": 13 } ] }
+{  "cid": 551,  "name": "Dorian Riggins",  "age": 85,  "address": {  "number": 9563,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Music", "Cigars", "Cigars", "Cooking" ],  "children": [ {  "name": "Lorine Riggins",  "age": 51 }, {  "name": "Sung Riggins" }, {  "name": "Fletcher Riggins",  "age": 60 }, {  "name": "Deon Riggins" } ] }
+{  "cid": 878,  "name": "Migdalia Bisker",  "age": 50,  "address": {  "number": 6699,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Computers", "Basketball" ],  "children": [ {  "name": "Moira Bisker" }, {  "name": "Tanisha Bisker" } ] }
+{  "cid": 412,  "name": "Devon Szalai",  "age": 26,  "address": {  "number": 2384,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Bass", "Books", "Books" ],  "children": [ {  "name": "Yolonda Szalai" }, {  "name": "Denita Szalai" }, {  "name": "Priscila Szalai",  "age": 10 }, {  "name": "Cassondra Szalai",  "age": 12 } ] }
+{  "cid": 608,  "name": "Bruce Stanley",  "age": 39,  "address": {  "number": 4532,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Tennis" ],  "children": [  ] }
+{  "cid": 743,  "name": "Nona Debroux",  "interests": [ "Bass" ],  "children": [  ] }
+{  "cid": 318,  "name": "Shaunna Royal",  "age": 86,  "address": {  "number": 8681,  "street": "7th St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Shantell Royal",  "age": 37 }, {  "name": "Shalon Royal",  "age": 50 }, {  "name": "Chung Royal",  "age": 26 } ] }
+{  "cid": 614,  "name": "Wallace Chaidy",  "interests": [ "Bass", "Movies", "Music" ],  "children": [ {  "name": "Refugio Chaidy" }, {  "name": "Hae Chaidy",  "age": 55 }, {  "name": "Julian Chaidy" }, {  "name": "Tabatha Chaidy" } ] }
+{  "cid": 390,  "name": "Shera Cung",  "age": 69,  "address": {  "number": 5850,  "street": "Hill St.",  "city": "San Jose" },  "interests": [ "Fishing", "Computers", "Cigars", "Base Jumping" ],  "children": [ {  "name": "Lenore Cung",  "age": 20 } ] }
+{  "cid": 518,  "name": "Cora Ingargiola",  "interests": [ "Skiing", "Squash", "Movies" ],  "children": [ {  "name": "Katlyn Ingargiola" }, {  "name": "Mike Ingargiola" }, {  "name": "Lawrence Ingargiola" }, {  "name": "Isabelle Ingargiola" } ] }
+{  "cid": 93,  "name": "Garth Raigosa",  "interests": [ "Basketball" ],  "children": [  ] }
+{  "cid": 41,  "name": "Kevin Giottonini",  "interests": [ "Skiing", "Bass" ],  "children": [ {  "name": "Victor Giottonini",  "age": 37 }, {  "name": "Alverta Giottonini" } ] }
+{  "cid": 7,  "name": "Karie Kaehler",  "age": 59,  "address": {  "number": 9875,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Computers", "Skiing", "Basketball", "Movies" ],  "children": [ {  "name": "Spring Kaehler",  "age": 17 } ] }
+{  "cid": 925,  "name": "Quintin Kizzie",  "interests": [ "Computers", "Tennis", "Bass", "Movies" ],  "children": [ {  "name": "Julius Kizzie",  "age": 11 }, {  "name": "Melissia Kizzie" }, {  "name": "Olga Kizzie",  "age": 42 } ] }
+{  "cid": 819,  "name": "Twanna Finnley",  "interests": [ "Squash", "Cigars" ],  "children": [ {  "name": "Reba Finnley" }, {  "name": "Moises Finnley" } ] }
+{  "cid": 162,  "name": "Chang Reek",  "age": 85,  "address": {  "number": 5943,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Tennis", "Movies" ],  "children": [ {  "name": "Camelia Reek" }, {  "name": "Eleonora Reek",  "age": 36 }, {  "name": "Shalonda Reek",  "age": 39 }, {  "name": "Stefan Reek",  "age": 64 } ] }
+{  "cid": 719,  "name": "Antoinette Boursiquot",  "age": 47,  "address": {  "number": 3652,  "street": "Cedar St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Dennis Boursiquot" }, {  "name": "Katelyn Boursiquot" }, {  "name": "Gabrielle Boursiquot" }, {  "name": "Deidre Boursiquot" } ] }
+{  "cid": 205,  "name": "Moises Plake",  "interests": [ "Puzzles", "Computers" ],  "children": [  ] }
+{  "cid": 319,  "name": "Ashlie Rott",  "age": 42,  "address": {  "number": 366,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Computers", "Cooking", "Databases" ],  "children": [  ] }
+{  "cid": 52,  "name": "Janna Tish",  "age": 12,  "address": {  "number": 2598,  "street": "Washington St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Mackenzie Tish" }, {  "name": "Ettie Tish" }, {  "name": "Hortencia Tish" }, {  "name": "Paul Tish" } ] }
+{  "cid": 903,  "name": "Elise Morenz",  "age": 17,  "address": {  "number": 8968,  "street": "View St.",  "city": "Mountain View" },  "interests": [  ],  "children": [  ] }
+{  "cid": 477,  "name": "Onie Kasica",  "age": 72,  "address": {  "number": 7963,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Skiing", "Bass", "Movies", "Skiing" ],  "children": [ {  "name": "Hallie Kasica",  "age": 44 } ] }
+{  "cid": 30,  "name": "Deedee Centner",  "interests": [ "Skiing", "Wine", "Databases", "Movies" ],  "children": [ {  "name": "Lorilee Centner",  "age": 30 }, {  "name": "Thad Centner" } ] }
+{  "cid": 434,  "name": "Tamesha Soho",  "age": 33,  "address": {  "number": 4534,  "street": "Park St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Cody Soho" }, {  "name": "Glennie Soho",  "age": 22 } ] }
+{  "cid": 158,  "name": "Rosalva Harvath",  "age": 84,  "address": {  "number": 5569,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Puzzles", "Wine", "Skiing", "Coffee" ],  "children": [ {  "name": "Taneka Harvath" }, {  "name": "Ina Harvath",  "age": 54 }, {  "name": "Joanne Harvath",  "age": 51 } ] }
+{  "cid": 246,  "name": "Kenda Heikkinen",  "age": 63,  "address": {  "number": 8924,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Databases" ],  "children": [  ] }
+{  "cid": 855,  "name": "Rosette Reen",  "age": 57,  "address": {  "number": 2767,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Basketball" ],  "children": [  ] }
+{  "cid": 484,  "name": "Bennie Dragaj",  "interests": [ "Fishing", "Databases", "Wine" ],  "children": [ {  "name": "Viva Dragaj",  "age": 13 } ] }
+{  "cid": 292,  "name": "Mariana Cosselman",  "interests": [ "Squash" ],  "children": [ {  "name": "Madge Cosselman",  "age": 43 } ] }
+{  "cid": 763,  "name": "Candis Deya",  "interests": [ "Computers" ],  "children": [ {  "name": "Lise Deya" }, {  "name": "Jeni Deya",  "age": 52 }, {  "name": "Domonique Deya",  "age": 24 }, {  "name": "Rubie Deya" } ] }
+{  "cid": 163,  "name": "Marcelene Sparano",  "age": 36,  "address": {  "number": 5722,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Basketball", "Databases" ],  "children": [ {  "name": "Luz Sparano" }, {  "name": "Cassandra Sparano",  "age": 21 }, {  "name": "Martina Sparano",  "age": 21 }, {  "name": "Elisabeth Sparano" } ] }
+{  "cid": 260,  "name": "Hedwig Caminero",  "age": 81,  "address": {  "number": 4305,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Video Games", "Databases" ],  "children": [ {  "name": "Hal Caminero" }, {  "name": "Cierra Caminero",  "age": 32 } ] }
+{  "cid": 650,  "name": "Darrin Orengo",  "interests": [  ],  "children": [ {  "name": "Linwood Orengo",  "age": 39 } ] }
+{  "cid": 762,  "name": "Towanda Yamat",  "interests": [ "Coffee", "Books", "Squash", "Bass" ],  "children": [ {  "name": "Michiko Yamat",  "age": 10 }, {  "name": "Ladonna Yamat" }, {  "name": "Brenton Yamat" } ] }
+{  "cid": 921,  "name": "Mario Nolden",  "age": 17,  "address": {  "number": 3977,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Gertrude Nolden" }, {  "name": "Ray Nolden" }, {  "name": "Inocencia Nolden" } ] }
+{  "cid": 672,  "name": "Pamelia Repka",  "age": 30,  "address": {  "number": 8837,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Coffee", "Base Jumping" ],  "children": [ {  "name": "Klara Repka",  "age": 19 }, {  "name": "Bennett Repka" }, {  "name": "Randy Repka",  "age": 13 }, {  "name": "Ervin Repka" } ] }
+{  "cid": 973,  "name": "Blanche Scivally",  "interests": [ "Movies", "Running", "Video Games", "Books" ],  "children": [ {  "name": "Josefina Scivally",  "age": 43 }, {  "name": "Joey Scivally",  "age": 34 } ] }
+{  "cid": 345,  "name": "Derick Rippel",  "age": 79,  "address": {  "number": 6843,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Running", "Basketball", "Computers", "Basketball" ],  "children": [  ] }
+{  "cid": 933,  "name": "Eartha Hershberger",  "age": 81,  "address": {  "number": 7013,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Waneta Hershberger" }, {  "name": "Katherine Hershberger",  "age": 67 }, {  "name": "Johnnie Hershberger",  "age": 25 }, {  "name": "Jovan Hershberger",  "age": 30 } ] }
+{  "cid": 109,  "name": "Rosette Simco",  "age": 79,  "address": {  "number": 5927,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Cooking", "Puzzles", "Basketball", "Skiing" ],  "children": [ {  "name": "Claudia Simco",  "age": 57 }, {  "name": "Altagracia Simco" } ] }
+{  "cid": 76,  "name": "Opal Blewett",  "interests": [ "Running", "Coffee", "Fishing" ],  "children": [ {  "name": "Violette Blewett" } ] }
+{  "cid": 250,  "name": "Angeles Saltonstall",  "interests": [ "Tennis", "Fishing", "Movies" ],  "children": [ {  "name": "Suzanna Saltonstall" } ] }
+{  "cid": 103,  "name": "Rosamond Milera",  "interests": [ "Cigars" ],  "children": [  ] }
+{  "cid": 328,  "name": "Mallory Sheffey",  "age": 27,  "address": {  "number": 8532,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Cooking" ],  "children": [ {  "name": "Regan Sheffey",  "age": 14 } ] }
+{  "cid": 287,  "name": "Cheryle Protano",  "interests": [ "Walking", "Coffee", "Puzzles", "Coffee" ],  "children": [ {  "name": "Karine Protano",  "age": 41 }, {  "name": "Mafalda Protano",  "age": 31 } ] }
+{  "cid": 928,  "name": "Maddie Diclaudio",  "age": 33,  "address": {  "number": 4674,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Base Jumping", "Databases", "Bass" ],  "children": [ {  "name": "Dominique Diclaudio",  "age": 12 } ] }
+{  "cid": 321,  "name": "Lidia Cicatello",  "interests": [ "Bass", "Movies", "Cooking", "Wine" ],  "children": [  ] }
+{  "cid": 977,  "name": "Ferdinand Barchick",  "age": 58,  "address": {  "number": 8278,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles", "Basketball", "Bass", "Fishing" ],  "children": [ {  "name": "Margeret Barchick",  "age": 32 }, {  "name": "Dwana Barchick" }, {  "name": "Kathryn Barchick" }, {  "name": "Tam Barchick" } ] }
+{  "cid": 642,  "name": "Odell Nova",  "age": 25,  "address": {  "number": 896,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Video Games", "Squash", "Music" ],  "children": [ {  "name": "Leopoldo Nova" }, {  "name": "Rickey Nova" }, {  "name": "Mike Nova",  "age": 14 }, {  "name": "Tamie Nova",  "age": 14 } ] }
+{  "cid": 288,  "name": "Sharice Bachicha",  "interests": [  ],  "children": [  ] }
+{  "cid": 261,  "name": "Aubrey Smulik",  "interests": [ "Music", "Coffee", "Base Jumping", "Fishing" ],  "children": [  ] }
+{  "cid": 395,  "name": "Bob Layman",  "age": 61,  "address": {  "number": 3646,  "street": "Washington St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [  ] }
+{  "cid": 180,  "name": "Theda Hilz",  "age": 35,  "address": {  "number": 9918,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Ethan Hilz" }, {  "name": "Bill Hilz",  "age": 12 } ] }
+{  "cid": 814,  "name": "Harriette Kasmarek",  "age": 68,  "address": {  "number": 7191,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Music", "Skiing" ],  "children": [ {  "name": "Melani Kasmarek",  "age": 24 }, {  "name": "Jesica Kasmarek",  "age": 22 } ] }
+{  "cid": 135,  "name": "Josette Dries",  "interests": [ "Base Jumping", "Movies" ],  "children": [ {  "name": "Ben Dries",  "age": 36 }, {  "name": "Wm Dries",  "age": 29 } ] }
+{  "cid": 542,  "name": "Eveline Smedley",  "age": 50,  "address": {  "number": 5513,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Skiing", "Walking" ],  "children": [ {  "name": "Lynsey Smedley",  "age": 26 } ] }
+{  "cid": 200,  "name": "Stacey Bertran",  "age": 78,  "address": {  "number": 9050,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Eugenia Bertran",  "age": 59 }, {  "name": "Lorri Bertran",  "age": 29 }, {  "name": "Corrie Bertran",  "age": 52 } ] }
+{  "cid": 535,  "name": "Juana Hirliman",  "age": 87,  "address": {  "number": 6763,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Movies" ],  "children": [ {  "name": "Ursula Hirliman",  "age": 40 }, {  "name": "Doretha Hirliman",  "age": 30 }, {  "name": "Leisha Hirliman",  "age": 49 } ] }
+{  "cid": 251,  "name": "Janeen Galston",  "interests": [ "Basketball", "Base Jumping" ],  "children": [  ] }
+{  "cid": 939,  "name": "Iris Moore",  "age": 23,  "address": {  "number": 8122,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Books", "Skiing", "Basketball", "Coffee" ],  "children": [ {  "name": "Consuela Moore" }, {  "name": "Delsie Moore" }, {  "name": "Stefan Moore",  "age": 11 } ] }
+{  "cid": 626,  "name": "Sydney Josten",  "age": 44,  "address": {  "number": 4815,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Cigars" ],  "children": [ {  "name": "Basil Josten",  "age": 14 }, {  "name": "Yasuko Josten" } ] }
+{  "cid": 438,  "name": "Allegra Pefanis",  "interests": [ "Computers", "Music", "Cigars" ],  "children": [  ] }
+{  "cid": 441,  "name": "Jamison Reeser",  "age": 84,  "address": {  "number": 9376,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Tennis" ],  "children": [ {  "name": "Elena Reeser",  "age": 28 } ] }
+{  "cid": 800,  "name": "Karon Johnsen",  "interests": [ "Movies" ],  "children": [ {  "name": "Roselee Johnsen",  "age": 25 } ] }
+{  "cid": 516,  "name": "Taunya Berkbigler",  "age": 82,  "address": {  "number": 5441,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Databases", "Tennis" ],  "children": [ {  "name": "Cherry Berkbigler",  "age": 27 }, {  "name": "Perry Berkbigler" } ] }
+{  "cid": 847,  "name": "Ashton Korba",  "age": 25,  "address": {  "number": 6450,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Cigars", "Computers", "Walking", "Video Games" ],  "children": [  ] }
+{  "cid": 476,  "name": "Kai Saggese",  "interests": [ "Squash", "Puzzles", "Books", "Movies" ],  "children": [  ] }
+{  "cid": 300,  "name": "Garret Colgrove",  "age": 85,  "address": {  "number": 9937,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Base Jumping", "Puzzles", "Fishing" ],  "children": [ {  "name": "Janna Colgrove" }, {  "name": "Jerilyn Colgrove",  "age": 35 } ] }
+{  "cid": 16,  "name": "Felisa Auletta",  "age": 55,  "address": {  "number": 7737,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Skiing", "Coffee", "Wine" ],  "children": [ {  "name": "Rosalia Auletta",  "age": 36 } ] }
+{  "cid": 23,  "name": "Micheal Konen",  "interests": [  ],  "children": [ {  "name": "Myong Konen",  "age": 26 }, {  "name": "Celinda Konen",  "age": 33 }, {  "name": "Tammy Konen",  "age": 53 }, {  "name": "Chester Konen" } ] }
+{  "cid": 580,  "name": "Liana Gabbert",  "interests": [ "Coffee", "Tennis", "Bass", "Running" ],  "children": [  ] }
+{  "cid": 341,  "name": "Francene Deats",  "interests": [ "Walking", "Databases", "Cigars", "Bass" ],  "children": [ {  "name": "Caron Deats" }, {  "name": "Geralyn Deats" }, {  "name": "Darell Deats" } ] }
+{  "cid": 323,  "name": "Rebeca Grisostomo",  "age": 26,  "address": {  "number": 399,  "street": "View St.",  "city": "Portland" },  "interests": [ "Music" ],  "children": [ {  "name": "Iva Grisostomo",  "age": 12 }, {  "name": "Ha Grisostomo" }, {  "name": "Lorna Grisostomo" } ] }
+{  "cid": 410,  "name": "Jennie Longhenry",  "age": 82,  "address": {  "number": 7427,  "street": "Main St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Charles Longhenry",  "age": 61 }, {  "name": "Faviola Longhenry",  "age": 25 }, {  "name": "Darline Longhenry" }, {  "name": "Lorean Longhenry" } ] }
+{  "cid": 4,  "name": "Bernita Gungor",  "age": 87,  "address": {  "number": 1208,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Walking" ],  "children": [ {  "name": "Valencia Gungor",  "age": 72 }, {  "name": "Evangeline Gungor",  "age": 76 }, {  "name": "Odell Gungor" }, {  "name": "Denny Gungor" } ] }
+{  "cid": 204,  "name": "Londa Herdt",  "interests": [  ],  "children": [ {  "name": "Marnie Herdt",  "age": 47 } ] }
+{  "cid": 270,  "name": "Lavon Ascenzo",  "interests": [ "Books", "Skiing" ],  "children": [  ] }
+{  "cid": 607,  "name": "Bert Garigliano",  "age": 71,  "address": {  "number": 3881,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Walking", "Wine" ],  "children": [ {  "name": "Junior Garigliano",  "age": 42 }, {  "name": "Willa Garigliano",  "age": 21 }, {  "name": "Carlo Garigliano" } ] }
+{  "cid": 87,  "name": "Torie Horuath",  "age": 21,  "address": {  "number": 2713,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Coffee", "Puzzles", "Cigars", "Walking" ],  "children": [ {  "name": "Joshua Horuath",  "age": 10 } ] }
+{  "cid": 742,  "name": "Andy Schifo",  "age": 36,  "address": {  "number": 4422,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Basketball" ],  "children": [  ] }
+{  "cid": 840,  "name": "Delicia Devoy",  "interests": [ "Fishing", "Running", "Skiing", "Video Games" ],  "children": [ {  "name": "Chan Devoy",  "age": 20 }, {  "name": "Bobbi Devoy",  "age": 30 }, {  "name": "Alyse Devoy",  "age": 40 } ] }
+{  "cid": 810,  "name": "Myron Dumlao",  "interests": [ "Wine", "Coffee" ],  "children": [ {  "name": "Josie Dumlao",  "age": 36 } ] }
+{  "cid": 446,  "name": "Lilly Grannell",  "age": 21,  "address": {  "number": 5894,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Computers", "Tennis", "Puzzles", "Books" ],  "children": [ {  "name": "Victor Grannell" } ] }
+{  "cid": 940,  "name": "Kitty Nalepka",  "interests": [ "Movies", "Wine", "Basketball" ],  "children": [ {  "name": "Kendra Nalepka" } ] }
+{  "cid": 634,  "name": "Katherina Parzych",  "interests": [  ],  "children": [ {  "name": "Modesta Parzych" }, {  "name": "Darin Parzych",  "age": 20 } ] }
+{  "cid": 493,  "name": "Lindsey Trout",  "age": 86,  "address": {  "number": 7619,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Base Jumping", "Skiing" ],  "children": [ {  "name": "Madlyn Trout",  "age": 58 }, {  "name": "Amie Trout",  "age": 72 } ] }
+{  "cid": 802,  "name": "Sang Hollman",  "interests": [ "Skiing" ],  "children": [ {  "name": "Carman Hollman" }, {  "name": "Kirstie Hollman",  "age": 40 }, {  "name": "Jacquetta Hollman" } ] }
+{  "cid": 803,  "name": "Yolonda Korf",  "interests": [ "Bass", "Skiing", "Music" ],  "children": [ {  "name": "Ivette Korf" }, {  "name": "Lashon Korf" } ] }
+{  "cid": 459,  "name": "Mable Ellwein",  "age": 60,  "address": {  "number": 1138,  "street": "Lake St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Stan Ellwein",  "age": 19 }, {  "name": "Ashlea Ellwein",  "age": 13 }, {  "name": "Tiesha Ellwein",  "age": 28 } ] }
+{  "cid": 154,  "name": "Jonelle Jephson",  "age": 39,  "address": {  "number": 2855,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Movies", "Basketball", "Tennis", "Base Jumping" ],  "children": [  ] }
+{  "cid": 959,  "name": "Hazel Haydon",  "age": 86,  "address": {  "number": 4530,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Tennis", "Cigars", "Squash", "Basketball" ],  "children": [ {  "name": "Micki Haydon",  "age": 33 }, {  "name": "Hollis Haydon",  "age": 23 }, {  "name": "Sonny Haydon",  "age": 71 } ] }
+{  "cid": 471,  "name": "Nicol Majersky",  "interests": [ "Video Games", "Books" ],  "children": [ {  "name": "Alise Majersky" }, {  "name": "Kathline Majersky",  "age": 53 }, {  "name": "Charlie Majersky",  "age": 45 }, {  "name": "Helaine Majersky" } ] }
+{  "cid": 999,  "name": "Bo Chaim",  "age": 59,  "address": {  "number": 8050,  "street": "View St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Zandra Chaim",  "age": 42 }, {  "name": "Theda Chaim",  "age": 14 }, {  "name": "Sharika Chaim",  "age": 22 } ] }
+{  "cid": 622,  "name": "Telma Rives",  "interests": [ "Basketball" ],  "children": [ {  "name": "Maribeth Rives",  "age": 42 }, {  "name": "Youlanda Rives",  "age": 13 }, {  "name": "Trang Rives" }, {  "name": "Hyun Rives" } ] }
+{  "cid": 679,  "name": "Maggie Kribs",  "age": 78,  "address": {  "number": 2846,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Video Games", "Books", "Databases", "Tennis" ],  "children": [ {  "name": "Estell Kribs",  "age": 54 }, {  "name": "Ranae Kribs",  "age": 54 }, {  "name": "Jalisa Kribs" } ] }
+{  "cid": 998,  "name": "Barry Schmaus",  "age": 65,  "address": {  "number": 4894,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Ma Schmaus",  "age": 40 }, {  "name": "Lashawn Schmaus",  "age": 13 }, {  "name": "Georgianne Schmaus",  "age": 38 } ] }
+{  "cid": 206,  "name": "Armand Hauersperger",  "age": 67,  "address": {  "number": 7266,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Wine" ],  "children": [ {  "name": "Charlott Hauersperger",  "age": 47 }, {  "name": "Kayla Hauersperger" }, {  "name": "Maris Hauersperger",  "age": 52 } ] }
+{  "cid": 424,  "name": "Camila Rightmire",  "age": 25,  "address": {  "number": 7542,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Bass", "Running", "Puzzles" ],  "children": [ {  "name": "Donny Rightmire",  "age": 14 }, {  "name": "Karlene Rightmire",  "age": 10 }, {  "name": "Nicholas Rightmire" }, {  "name": "Margareta Rightmire" } ] }
+{  "cid": 677,  "name": "Brigid Sarabia",  "age": 89,  "address": {  "number": 918,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Elisa Sarabia" }, {  "name": "Pura Sarabia",  "age": 56 } ] }
+{  "cid": 475,  "name": "Brinda Gouker",  "interests": [  ],  "children": [ {  "name": "Gayle Gouker",  "age": 52 } ] }
+{  "cid": 683,  "name": "Dodie Crall",  "age": 37,  "address": {  "number": 1337,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Wine" ],  "children": [ {  "name": "Cassy Crall" }, {  "name": "Thu Crall",  "age": 19 } ] }
+{  "cid": 869,  "name": "Lino Wooderson",  "interests": [  ],  "children": [ {  "name": "Nola Wooderson" }, {  "name": "Leticia Wooderson",  "age": 36 }, {  "name": "Bernardine Wooderson" } ] }
+{  "cid": 400,  "name": "Jeffery Maresco",  "interests": [ "Coffee", "Bass" ],  "children": [  ] }
+{  "cid": 756,  "name": "Marisol Noyes",  "interests": [  ],  "children": [ {  "name": "Delora Noyes" }, {  "name": "Jonelle Noyes",  "age": 44 } ] }
+{  "cid": 464,  "name": "Petra Kinsel",  "interests": [ "Wine" ],  "children": [ {  "name": "Janise Kinsel" }, {  "name": "Donnie Kinsel",  "age": 26 }, {  "name": "Joana Kinsel",  "age": 12 } ] }
+{  "cid": 73,  "name": "Kelsey Flever",  "age": 20,  "address": {  "number": 3555,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Tennis", "Puzzles", "Video Games" ],  "children": [ {  "name": "Isis Flever" }, {  "name": "Gonzalo Flever" } ] }
+{  "cid": 333,  "name": "Conchita Olivera",  "age": 37,  "address": {  "number": 8519,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Trenton Olivera" }, {  "name": "Shin Olivera",  "age": 26 }, {  "name": "Everett Olivera",  "age": 15 }, {  "name": "Shera Olivera",  "age": 20 } ] }
+{  "cid": 51,  "name": "Simonne Cape",  "interests": [ "Bass", "Bass", "Books" ],  "children": [ {  "name": "Leland Cape" }, {  "name": "Gearldine Cape" } ] }
+{  "cid": 414,  "name": "Sixta Smithheart",  "interests": [ "Skiing", "Books", "Computers" ],  "children": [ {  "name": "Nicholas Smithheart" } ] }
+{  "cid": 367,  "name": "Cassondra Fabiani",  "interests": [ "Squash", "Tennis" ],  "children": [ {  "name": "Evia Fabiani" }, {  "name": "Chaya Fabiani" }, {  "name": "Sherman Fabiani" }, {  "name": "Kathi Fabiani",  "age": 54 } ] }
+{  "cid": 806,  "name": "Corliss Sharratt",  "interests": [ "Basketball", "Cigars", "Cooking" ],  "children": [ {  "name": "Albertine Sharratt" }, {  "name": "Nobuko Sharratt",  "age": 29 }, {  "name": "Neil Sharratt" } ] }
+{  "cid": 944,  "name": "Johana Hisman",  "interests": [ "Wine" ],  "children": [ {  "name": "Kirstin Hisman",  "age": 43 }, {  "name": "Darwin Hisman",  "age": 29 } ] }
+{  "cid": 850,  "name": "Garnet Younce",  "interests": [ "Databases", "Video Games", "Books" ],  "children": [ {  "name": "Syble Younce",  "age": 16 } ] }
+{  "cid": 253,  "name": "Rosaura Maitland",  "age": 71,  "address": {  "number": 6403,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Cigars", "Basketball", "Coffee", "Cigars" ],  "children": [ {  "name": "Letisha Maitland",  "age": 43 }, {  "name": "Margart Maitland",  "age": 13 }, {  "name": "Neal Maitland" }, {  "name": "Hayden Maitland" } ] }
+{  "cid": 911,  "name": "Eileen Bartolomeo",  "age": 20,  "address": {  "number": 8915,  "street": "Main St.",  "city": "Portland" },  "interests": [  ],  "children": [  ] }
+{  "cid": 166,  "name": "Gregorio Plummer",  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Santiago Plummer" }, {  "name": "Malisa Plummer",  "age": 59 }, {  "name": "Tracie Plummer",  "age": 40 }, {  "name": "Florentina Plummer",  "age": 23 } ] }
+{  "cid": 45,  "name": "Jarrod Ridener",  "interests": [ "Skiing", "Tennis", "Squash", "Puzzles" ],  "children": [  ] }
+{  "cid": 667,  "name": "Shaniqua Deist",  "interests": [ "Puzzles", "Books", "Cigars" ],  "children": [  ] }
+{  "cid": 385,  "name": "Jody Favaron",  "age": 73,  "address": {  "number": 4724,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Elane Favaron",  "age": 47 }, {  "name": "Katherine Favaron",  "age": 38 } ] }
+{  "cid": 160,  "name": "Yevette Chanez",  "interests": [ "Bass", "Wine", "Coffee" ],  "children": [ {  "name": "Walter Chanez",  "age": 11 }, {  "name": "Pa Chanez",  "age": 27 } ] }
+{  "cid": 208,  "name": "Mirta Kenison",  "age": 68,  "address": {  "number": 2880,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Base Jumping", "Cigars", "Skiing", "Fishing" ],  "children": [ {  "name": "Dinorah Kenison",  "age": 15 }, {  "name": "Roy Kenison" } ] }
+{  "cid": 42,  "name": "Asley Simco",  "age": 38,  "address": {  "number": 3322,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Fishing", "Running", "Cigars" ],  "children": [ {  "name": "Micheal Simco" }, {  "name": "Lawerence Simco" } ] }
+{  "cid": 396,  "name": "Delfina Calcara",  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Sybil Calcara" } ] }
+{  "cid": 67,  "name": "Tobie Mattan",  "interests": [  ],  "children": [  ] }
+{  "cid": 14,  "name": "Chance Nicoson",  "interests": [ "Tennis" ],  "children": [ {  "name": "Willette Nicoson",  "age": 39 }, {  "name": "Glennis Nicoson" }, {  "name": "Philip Nicoson" }, {  "name": "Cody Nicoson",  "age": 26 } ] }
+{  "cid": 363,  "name": "Merlene Hoying",  "age": 25,  "address": {  "number": 2105,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Squash", "Squash", "Music" ],  "children": [ {  "name": "Andrew Hoying",  "age": 10 } ] }
+{  "cid": 658,  "name": "Truman Leitner",  "interests": [ "Computers", "Bass", "Walking" ],  "children": [  ] }
+{  "cid": 78,  "name": "Wesley Huggler",  "age": 80,  "address": {  "number": 3078,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Base Jumping", "Movies", "Skiing" ],  "children": [ {  "name": "Chassidy Huggler" }, {  "name": "Emogene Huggler" }, {  "name": "Cheryle Huggler" } ] }
+{  "cid": 436,  "name": "Xenia Pool",  "interests": [ "Books" ],  "children": [  ] }
+{  "cid": 353,  "name": "Melody Bernas",  "age": 76,  "address": {  "number": 6783,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Kristel Bernas",  "age": 45 }, {  "name": "Clorinda Bernas",  "age": 10 }, {  "name": "Natosha Bernas" } ] }
+{  "cid": 879,  "name": "Vinnie Antoniewicz",  "age": 45,  "address": {  "number": 1633,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Cooking", "Puzzles" ],  "children": [  ] }
+{  "cid": 449,  "name": "Jacinda Markle",  "interests": [ "Basketball", "Basketball", "Computers" ],  "children": [ {  "name": "Tam Markle",  "age": 45 } ] }
+{  "cid": 728,  "name": "Bruno Freeburger",  "age": 84,  "address": {  "number": 2482,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Computers" ],  "children": [ {  "name": "Shizuko Freeburger" } ] }
+{  "cid": 488,  "name": "Dannielle Wilkie",  "interests": [ "Running", "Fishing", "Coffee", "Basketball" ],  "children": [ {  "name": "Vita Wilkie",  "age": 17 }, {  "name": "Marisa Wilkie" }, {  "name": "Faustino Wilkie" } ] }
+{  "cid": 479,  "name": "Danilo Varney",  "age": 17,  "address": {  "number": 9330,  "street": "Hill St.",  "city": "Portland" },  "interests": [ "Wine" ],  "children": [ {  "name": "Shelby Varney" }, {  "name": "Fidela Varney" }, {  "name": "Maynard Varney" }, {  "name": "Lindsay Varney" } ] }
+{  "cid": 401,  "name": "Moises Jago",  "age": 27,  "address": {  "number": 3773,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Music" ],  "children": [ {  "name": "Shoshana Jago" }, {  "name": "Juliet Jago" }, {  "name": "Berneice Jago",  "age": 13 } ] }
+{  "cid": 338,  "name": "Dorthey Roncskevitz",  "age": 38,  "address": {  "number": 4366,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Computers" ],  "children": [ {  "name": "Mindy Roncskevitz" } ] }
+{  "cid": 445,  "name": "Walton Komo",  "age": 16,  "address": {  "number": 8769,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Running", "Basketball", "Tennis" ],  "children": [  ] }
+{  "cid": 691,  "name": "Sharee Charrier",  "age": 17,  "address": {  "number": 6693,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Puzzles", "Cooking", "Bass" ],  "children": [ {  "name": "Odessa Charrier" } ] }
+{  "cid": 271,  "name": "Carey Ronin",  "age": 44,  "address": {  "number": 8141,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Cigars", "Video Games" ],  "children": [ {  "name": "Lonny Ronin" }, {  "name": "Armanda Ronin" } ] }
+{  "cid": 63,  "name": "Mayra Hait",  "interests": [ "Cigars", "Cigars", "Bass", "Books" ],  "children": [  ] }
+{  "cid": 274,  "name": "Claude Harral",  "interests": [ "Squash", "Bass", "Cooking" ],  "children": [ {  "name": "Archie Harral" }, {  "name": "Royal Harral" } ] }
+{  "cid": 335,  "name": "Odessa Dammeyer",  "age": 18,  "address": {  "number": 6828,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Basketball", "Bass", "Cigars" ],  "children": [ {  "name": "Lindsey Dammeyer" } ] }
+{  "cid": 985,  "name": "Arnette Farlow",  "age": 23,  "address": {  "number": 7843,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Running", "Databases" ],  "children": [ {  "name": "Lora Farlow",  "age": 12 }, {  "name": "Arlen Farlow",  "age": 11 }, {  "name": "Rodney Farlow" }, {  "name": "Tori Farlow",  "age": 11 } ] }
+{  "cid": 786,  "name": "Johnsie Maheux",  "interests": [ "Cigars" ],  "children": [ {  "name": "Danuta Maheux" } ] }
+{  "cid": 528,  "name": "Tamela Witherbee",  "interests": [  ],  "children": [ {  "name": "Penney Witherbee" } ] }
+{  "cid": 186,  "name": "Krystle Spangler",  "age": 15,  "address": {  "number": 4697,  "street": "Cedar St.",  "city": "Seattle" },  "interests": [ "Cigars", "Squash", "Coffee", "Video Games" ],  "children": [  ] }
+{  "cid": 721,  "name": "Jesica Tinder",  "age": 28,  "address": {  "number": 5526,  "street": "7th St.",  "city": "Mountain View" },  "interests": [  ],  "children": [  ] }
+{  "cid": 415,  "name": "Valentin Mclarney",  "interests": [ "Squash", "Squash", "Video Games" ],  "children": [ {  "name": "Vanda Mclarney",  "age": 17 } ] }
+{  "cid": 805,  "name": "Gaylord Ginder",  "interests": [ "Databases", "Coffee" ],  "children": [ {  "name": "Lucina Ginder" }, {  "name": "Harriett Ginder" } ] }
+{  "cid": 142,  "name": "Ervin Softleigh",  "interests": [ "Computers", "Skiing", "Cooking", "Coffee" ],  "children": [ {  "name": "Russell Softleigh",  "age": 50 }, {  "name": "Kristy Softleigh",  "age": 54 }, {  "name": "Refugio Softleigh" } ] }
+{  "cid": 962,  "name": "Taryn Coley",  "interests": [ "Running", "Basketball", "Cooking" ],  "children": [  ] }
+{  "cid": 935,  "name": "Sharita Aspegren",  "interests": [  ],  "children": [ {  "name": "Russell Aspegren",  "age": 35 }, {  "name": "Bernardina Aspegren" }, {  "name": "Isobel Aspegren",  "age": 11 }, {  "name": "Reva Aspegren" } ] }
+{  "cid": 995,  "name": "Kiersten Basila",  "interests": [  ],  "children": [ {  "name": "Norman Basila",  "age": 17 }, {  "name": "Reginia Basila" }, {  "name": "Gilberto Basila" }, {  "name": "Elvira Basila",  "age": 49 } ] }
+{  "cid": 884,  "name": "Laila Marta",  "interests": [ "Fishing", "Movies" ],  "children": [ {  "name": "Carlota Marta",  "age": 19 } ] }
+{  "cid": 704,  "name": "Melodee Clemons",  "interests": [ "Base Jumping", "Tennis", "Video Games" ],  "children": [ {  "name": "Doreatha Clemons",  "age": 22 } ] }
+{  "cid": 680,  "name": "Domenica Qunnarath",  "interests": [  ],  "children": [  ] }
+{  "cid": 138,  "name": "Ora Villafane",  "interests": [ "Walking", "Cooking" ],  "children": [ {  "name": "Deeann Villafane",  "age": 22 }, {  "name": "Cody Villafane",  "age": 47 } ] }
+{  "cid": 55,  "name": "Terrence Bryant",  "age": 12,  "address": {  "number": 3188,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Wine", "Cooking" ],  "children": [ {  "name": "Dayna Bryant" } ] }
+{  "cid": 562,  "name": "Etta Hooton",  "interests": [ "Databases", "Cigars", "Music", "Video Games" ],  "children": [ {  "name": "Sherice Hooton" }, {  "name": "Estefana Hooton",  "age": 38 }, {  "name": "Nidia Hooton",  "age": 47 }, {  "name": "Erwin Hooton" } ] }
+{  "cid": 827,  "name": "Clementina Papin",  "interests": [ "Music", "Basketball", "Cigars" ],  "children": [ {  "name": "Catina Papin" }, {  "name": "Demetrius Papin",  "age": 59 }, {  "name": "Marylou Papin",  "age": 12 }, {  "name": "Apryl Papin",  "age": 16 } ] }
+{  "cid": 256,  "name": "Chester Rosenberg",  "age": 46,  "address": {  "number": 8673,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Basketball" ],  "children": [ {  "name": "Gemma Rosenberg" }, {  "name": "Marty Rosenberg" } ] }
+{  "cid": 628,  "name": "Tomoko Alcantara",  "age": 56,  "address": {  "number": 3556,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Running", "Tennis" ],  "children": [ {  "name": "Babara Alcantara",  "age": 31 }, {  "name": "Ilana Alcantara" }, {  "name": "Maren Alcantara",  "age": 45 } ] }
+{  "cid": 696,  "name": "Nadia Dunklee",  "interests": [  ],  "children": [ {  "name": "Mendy Dunklee",  "age": 17 }, {  "name": "Edgar Dunklee" }, {  "name": "Pasquale Dunklee" }, {  "name": "Colin Dunklee" } ] }
+{  "cid": 760,  "name": "Karena Romp",  "interests": [ "Cigars", "Databases", "Squash", "Tennis" ],  "children": [ {  "name": "Donn Romp" }, {  "name": "Antonio Romp" }, {  "name": "Kattie Romp",  "age": 54 }, {  "name": "Marylynn Romp",  "age": 53 } ] }
+{  "cid": 428,  "name": "Tiffany Waye",  "interests": [ "Basketball", "Cigars" ],  "children": [ {  "name": "Berna Waye" }, {  "name": "Kiersten Waye" }, {  "name": "Romeo Waye" }, {  "name": "Marvel Waye",  "age": 56 } ] }
+{  "cid": 458,  "name": "Ivan Sien",  "age": 17,  "address": {  "number": 9981,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Cooking", "Coffee" ],  "children": [ {  "name": "Laurence Sien" }, {  "name": "Nelle Sien" }, {  "name": "Thalia Sien" } ] }
+{  "cid": 776,  "name": "Dagmar Sarkis",  "interests": [ "Basketball", "Running", "Wine" ],  "children": [ {  "name": "Tari Sarkis" }, {  "name": "Rana Sarkis",  "age": 56 }, {  "name": "Merissa Sarkis" }, {  "name": "Lori Sarkis",  "age": 26 } ] }
+{  "cid": 381,  "name": "Kassandra Ereth",  "interests": [ "Base Jumping", "Base Jumping", "Databases", "Walking" ],  "children": [ {  "name": "Angelina Ereth",  "age": 46 }, {  "name": "Tristan Ereth" }, {  "name": "Johnny Ereth" } ] }
+{  "cid": 172,  "name": "Weldon Alquesta",  "interests": [ "Music", "Fishing", "Music" ],  "children": [ {  "name": "Kip Alquesta" } ] }
+{  "cid": 54,  "name": "Haywood Vasiloff",  "age": 63,  "address": {  "number": 8780,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Celsa Vasiloff",  "age": 40 }, {  "name": "Shawana Vasiloff",  "age": 43 }, {  "name": "Joel Vasiloff",  "age": 42 }, {  "name": "Timmy Vasiloff",  "age": 33 } ] }
+{  "cid": 675,  "name": "Camellia Brickett",  "interests": [ "Running" ],  "children": [ {  "name": "Leona Brickett" }, {  "name": "Mario Brickett" }, {  "name": "Nadine Brickett",  "age": 35 }, {  "name": "Marlon Brickett",  "age": 31 } ] }
+{  "cid": 177,  "name": "Wilda Hanisch",  "interests": [ "Wine", "Computers" ],  "children": [ {  "name": "Shannan Hanisch" }, {  "name": "Marissa Hanisch",  "age": 30 }, {  "name": "Keely Hanisch",  "age": 54 }, {  "name": "Humberto Hanisch",  "age": 17 } ] }
+{  "cid": 266,  "name": "Carlee Friddle",  "age": 74,  "address": {  "number": 6538,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Databases" ],  "children": [ {  "name": "Candie Friddle" }, {  "name": "Zoila Friddle",  "age": 59 } ] }
+{  "cid": 584,  "name": "Bailey Janes",  "interests": [  ],  "children": [ {  "name": "Marylou Janes" }, {  "name": "Andra Janes" } ] }
+{  "cid": 36,  "name": "Neoma Preist",  "age": 69,  "address": {  "number": 4830,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Databases", "Computers", "Coffee" ],  "children": [ {  "name": "Shery Preist" }, {  "name": "Kelvin Preist",  "age": 43 } ] }
+{  "cid": 417,  "name": "Irene Funderberg",  "age": 45,  "address": {  "number": 8503,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Music", "Skiing", "Running" ],  "children": [ {  "name": "Lyndia Funderberg",  "age": 14 }, {  "name": "Herta Funderberg" } ] }
+{  "cid": 282,  "name": "Emelda Dawood",  "age": 32,  "address": {  "number": 5261,  "street": "View St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Venus Dawood",  "age": 12 }, {  "name": "Gertrude Dawood" }, {  "name": "Yen Dawood" }, {  "name": "Theresa Dawood",  "age": 16 } ] }
+{  "cid": 31,  "name": "Venus Toboz",  "age": 44,  "address": {  "number": 9465,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Running" ],  "children": [ {  "name": "Ashlie Toboz" } ] }
+{  "cid": 920,  "name": "Mirtha Dellbringge",  "interests": [ "Walking", "Basketball", "Basketball" ],  "children": [ {  "name": "Morgan Dellbringge",  "age": 51 }, {  "name": "Alease Dellbringge",  "age": 35 } ] }
+{  "cid": 932,  "name": "Kraig Bomia",  "interests": [ "Music" ],  "children": [  ] }
+{  "cid": 269,  "name": "Dante Sharko",  "interests": [ "Base Jumping" ],  "children": [ {  "name": "Ahmad Sharko",  "age": 34 }, {  "name": "Mona Sharko" }, {  "name": "Stephaine Sharko",  "age": 42 }, {  "name": "Adrianna Sharko" } ] }
+{  "cid": 277,  "name": "Malena Smock",  "interests": [ "Running", "Base Jumping" ],  "children": [ {  "name": "Inocencia Smock",  "age": 50 }, {  "name": "Cleveland Smock" } ] }
+{  "cid": 807,  "name": "Maryanne Kuzminski",  "age": 21,  "address": {  "number": 1601,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Running" ],  "children": [ {  "name": "India Kuzminski" }, {  "name": "Adell Kuzminski" } ] }
+{  "cid": 515,  "name": "Connie Banis",  "interests": [ "Coffee" ],  "children": [ {  "name": "Brittni Banis" }, {  "name": "Deloras Banis",  "age": 25 } ] }
+{  "cid": 225,  "name": "Shantel Drapeaux",  "interests": [ "Databases" ],  "children": [ {  "name": "Felicidad Drapeaux" }, {  "name": "Wanetta Drapeaux",  "age": 52 }, {  "name": "Louise Drapeaux",  "age": 28 }, {  "name": "Pat Drapeaux" } ] }
+{  "cid": 741,  "name": "Lesia Risatti",  "age": 48,  "address": {  "number": 7378,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Fishing", "Wine", "Databases" ],  "children": [ {  "name": "Tangela Risatti" }, {  "name": "Leonel Risatti",  "age": 33 }, {  "name": "Cythia Risatti",  "age": 36 } ] }
+{  "cid": 8,  "name": "Audria Haylett",  "age": 44,  "address": {  "number": 4872,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Cooking", "Fishing", "Video Games" ],  "children": [ {  "name": "Lacie Haylett",  "age": 19 } ] }
+{  "cid": 33,  "name": "Rayford Velmontes",  "interests": [ "Fishing", "Video Games" ],  "children": [  ] }
+{  "cid": 674,  "name": "Alice Gurrola",  "interests": [ "Puzzles", "Skiing", "Video Games", "Computers" ],  "children": [ {  "name": "Lee Gurrola" } ] }
+{  "cid": 788,  "name": "Franklyn Crowner",  "age": 56,  "address": {  "number": 4186,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Base Jumping", "Base Jumping", "Books", "Computers" ],  "children": [ {  "name": "Adrian Crowner",  "age": 43 }, {  "name": "Vasiliki Crowner" } ] }
+{  "cid": 980,  "name": "Harley Lappe",  "age": 56,  "address": {  "number": 647,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Books", "Cigars", "Basketball" ],  "children": [ {  "name": "Maxwell Lappe" }, {  "name": "Gemma Lappe",  "age": 32 }, {  "name": "Ester Lappe",  "age": 40 }, {  "name": "Myles Lappe",  "age": 36 } ] }
+{  "cid": 712,  "name": "Jack Lamoreux",  "age": 32,  "address": {  "number": 4486,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Rubin Lamoreux",  "age": 15 }, {  "name": "Jonelle Lamoreux",  "age": 10 }, {  "name": "Shonna Lamoreux" }, {  "name": "India Lamoreux",  "age": 17 } ] }
+{  "cid": 455,  "name": "Manual Altizer",  "age": 70,  "address": {  "number": 6293,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Running", "Fishing", "Coffee" ],  "children": [ {  "name": "Katherine Altizer" } ] }
+{  "cid": 904,  "name": "Holley Tofil",  "age": 51,  "address": {  "number": 8946,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Music", "Squash" ],  "children": [ {  "name": "Kristal Tofil" } ] }
+{  "cid": 880,  "name": "Sara Abo",  "interests": [ "Squash" ],  "children": [  ] }
+{  "cid": 284,  "name": "Mason Fuel",  "interests": [ "Bass", "Tennis", "Computers", "Coffee" ],  "children": [ {  "name": "Odis Fuel" }, {  "name": "Sanjuanita Fuel" } ] }
+{  "cid": 496,  "name": "Lonna Starkweather",  "age": 80,  "address": {  "number": 1162,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Coffee", "Bass", "Running" ],  "children": [ {  "name": "Matilda Starkweather" } ] }
+{  "cid": 657,  "name": "Rory Teachman",  "interests": [  ],  "children": [  ] }
+{  "cid": 257,  "name": "Altha Jastrzebski",  "age": 21,  "address": {  "number": 4405,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Puzzles" ],  "children": [  ] }
+{  "cid": 450,  "name": "Althea Mohammed",  "interests": [ "Fishing", "Databases" ],  "children": [ {  "name": "Jasper Mohammed" } ] }
+{  "cid": 801,  "name": "Julio Brun",  "age": 13,  "address": {  "number": 9774,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles", "Running", "Puzzles", "Base Jumping" ],  "children": [ {  "name": "Peter Brun" }, {  "name": "Remona Brun" }, {  "name": "Giovanni Brun" } ] }
+{  "cid": 891,  "name": "Jesusita Bhatia",  "age": 57,  "address": {  "number": 1476,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Walking" ],  "children": [  ] }
+{  "cid": 520,  "name": "Janay Bernbeck",  "interests": [ "Databases", "Databases" ],  "children": [ {  "name": "Aurea Bernbeck" }, {  "name": "Tiara Bernbeck" }, {  "name": "Alfredia Bernbeck",  "age": 26 } ] }
+{  "cid": 505,  "name": "Mike Runk",  "interests": [ "Databases", "Computers", "Running", "Video Games" ],  "children": [ {  "name": "Lashawn Runk",  "age": 21 } ] }
+{  "cid": 876,  "name": "Chelsie Motten",  "interests": [ "Music", "Squash", "Music", "Walking" ],  "children": [ {  "name": "Nida Motten" }, {  "name": "Taneka Motten",  "age": 10 }, {  "name": "Maynard Motten",  "age": 57 } ] }
+{  "cid": 993,  "name": "Shawn Irie",  "interests": [ "Fishing", "Cigars" ],  "children": [ {  "name": "Tonette Irie" } ] }
+{  "cid": 934,  "name": "Dessie Lockmiller",  "age": 70,  "address": {  "number": 4313,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Coffee", "Puzzles" ],  "children": [  ] }
+{  "cid": 564,  "name": "Inger Dargin",  "age": 56,  "address": {  "number": 8704,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Wine", "Running", "Computers" ],  "children": [  ] }
+{  "cid": 587,  "name": "Santos Monterio",  "age": 36,  "address": {  "number": 4454,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Databases", "Music", "Cooking" ],  "children": [ {  "name": "Lashonda Monterio" } ] }
+{  "cid": 854,  "name": "Angie Oyster",  "age": 32,  "address": {  "number": 8860,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Coffee", "Movies", "Fishing" ],  "children": [ {  "name": "Hugh Oyster",  "age": 10 } ] }
+{  "cid": 275,  "name": "Natalie Ifeanyi",  "interests": [  ],  "children": [  ] }
+{  "cid": 398,  "name": "Piedad Paranada",  "interests": [  ],  "children": [ {  "name": "Claribel Paranada",  "age": 22 }, {  "name": "Lincoln Paranada" }, {  "name": "Cecilia Paranada" } ] }
+{  "cid": 585,  "name": "Young Drube",  "age": 21,  "address": {  "number": 6960,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Basketball", "Fishing", "Walking" ],  "children": [ {  "name": "Irwin Drube" }, {  "name": "Gustavo Drube" } ] }
+{  "cid": 15,  "name": "Berry Faubel",  "age": 55,  "address": {  "number": 2806,  "street": "Oak St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Tiffiny Faubel",  "age": 12 }, {  "name": "Hilaria Faubel",  "age": 19 }, {  "name": "Wesley Faubel",  "age": 37 }, {  "name": "Wei Faubel",  "age": 28 } ] }
+{  "cid": 695,  "name": "Wyatt Eveleth",  "age": 28,  "address": {  "number": 5421,  "street": "View St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Orval Eveleth" }, {  "name": "Beth Eveleth",  "age": 11 }, {  "name": "Yuki Eveleth" }, {  "name": "Alyse Eveleth",  "age": 14 } ] }
+{  "cid": 555,  "name": "Agustina Bretthauer",  "interests": [ "Cigars" ],  "children": [ {  "name": "Arthur Bretthauer",  "age": 33 }, {  "name": "Titus Bretthauer",  "age": 33 }, {  "name": "Margret Bretthauer" } ] }
+{  "cid": 796,  "name": "Daniele Brisk",  "interests": [ "Walking", "Bass" ],  "children": [  ] }
+{  "cid": 570,  "name": "Lee Basora",  "interests": [ "Squash", "Cigars" ],  "children": [  ] }
+{  "cid": 572,  "name": "Darcy Polycarpe",  "age": 35,  "address": {  "number": 8051,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Computers", "Coffee", "Walking", "Walking" ],  "children": [ {  "name": "Kenneth Polycarpe" } ] }
+{  "cid": 25,  "name": "Goldie Vanhandel",  "age": 37,  "address": {  "number": 6568,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Bass", "Fishing", "Cigars" ],  "children": [  ] }
+{  "cid": 895,  "name": "Joie Siffert",  "interests": [ "Wine", "Skiing", "Puzzles", "Tennis" ],  "children": [ {  "name": "Erma Siffert" }, {  "name": "Natosha Siffert",  "age": 38 }, {  "name": "Somer Siffert",  "age": 27 } ] }
+{  "cid": 403,  "name": "Kayleigh Houey",  "interests": [ "Fishing", "Music" ],  "children": [ {  "name": "Ta Houey" }, {  "name": "Ayana Houey" }, {  "name": "Dominique Houey" }, {  "name": "Denise Houey",  "age": 48 } ] }
+{  "cid": 173,  "name": "Annamae Lucien",  "age": 46,  "address": {  "number": 1253,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Puzzles", "Cooking", "Squash" ],  "children": [ {  "name": "Sanjuana Lucien",  "age": 21 }, {  "name": "Nathanael Lucien",  "age": 27 }, {  "name": "Jae Lucien" }, {  "name": "Judith Lucien" } ] }
+{  "cid": 490,  "name": "Valentine Dolecki",  "interests": [ "Video Games", "Video Games", "Bass", "Bass" ],  "children": [ {  "name": "Rene Dolecki" }, {  "name": "Omega Dolecki",  "age": 37 }, {  "name": "Hedwig Dolecki" } ] }
+{  "cid": 877,  "name": "Nicki Lipkind",  "interests": [ "Books", "Movies" ],  "children": [ {  "name": "Yahaira Lipkind",  "age": 12 } ] }
+{  "cid": 720,  "name": "Vannesa Prabel",  "interests": [ "Basketball", "Cigars", "Running", "Video Games" ],  "children": [ {  "name": "Carter Prabel",  "age": 23 }, {  "name": "Rodger Prabel",  "age": 48 }, {  "name": "Odilia Prabel" } ] }
+{  "cid": 237,  "name": "Sona Hehn",  "age": 47,  "address": {  "number": 3720,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Computers", "Squash", "Coffee" ],  "children": [ {  "name": "Marquerite Hehn" }, {  "name": "Suellen Hehn",  "age": 29 }, {  "name": "Herb Hehn",  "age": 29 } ] }
+{  "cid": 35,  "name": "Saundra Aparo",  "age": 86,  "address": {  "number": 9550,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Cigars", "Skiing", "Video Games", "Books" ],  "children": [  ] }
+{  "cid": 332,  "name": "Malcom Cafasso",  "interests": [  ],  "children": [ {  "name": "Marie Cafasso" }, {  "name": "Asley Cafasso",  "age": 38 } ] }
+{  "cid": 866,  "name": "Bonita Kauphusman",  "interests": [  ],  "children": [  ] }
+{  "cid": 223,  "name": "Margurite Embelton",  "age": 19,  "address": {  "number": 554,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Running", "Fishing" ],  "children": [ {  "name": "Sherie Embelton" }, {  "name": "Monica Embelton" }, {  "name": "Jeanne Embelton" }, {  "name": "Santiago Embelton" } ] }
+{  "cid": 825,  "name": "Kirstie Rinebold",  "age": 57,  "address": {  "number": 9463,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Cooking", "Cigars", "Books" ],  "children": [ {  "name": "Vonda Rinebold" }, {  "name": "Man Rinebold",  "age": 21 } ] }
+{  "cid": 336,  "name": "Jalisa Talamantez",  "age": 78,  "address": {  "number": 9902,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Video Games", "Squash" ],  "children": [  ] }
+{  "cid": 176,  "name": "Kellie Andruszkiewic",  "interests": [ "Fishing", "Puzzles", "Wine", "Skiing" ],  "children": [ {  "name": "Xiao Andruszkiewic" }, {  "name": "Al Andruszkiewic",  "age": 43 } ] }
+{  "cid": 833,  "name": "Lakisha Petkoff",  "interests": [ "Coffee" ],  "children": [ {  "name": "Brittanie Petkoff" }, {  "name": "Ashli Petkoff" } ] }
+{  "cid": 192,  "name": "Shakira Delmonte",  "age": 10,  "address": {  "number": 8838,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Books", "Cigars", "Bass", "Base Jumping" ],  "children": [ {  "name": "Sergio Delmonte" }, {  "name": "Aida Delmonte" }, {  "name": "Juliane Delmonte" } ] }
+{  "cid": 737,  "name": "Jeffrey Chesson",  "age": 13,  "address": {  "number": 6833,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Tennis", "Computers" ],  "children": [ {  "name": "Clayton Chesson" }, {  "name": "Yi Chesson" } ] }
+{  "cid": 567,  "name": "Peggie Madhavan",  "interests": [ "Computers", "Bass" ],  "children": [  ] }
+{  "cid": 286,  "name": "Tara Sioma",  "age": 18,  "address": {  "number": 9425,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Dawna Sioma" }, {  "name": "Jeanne Sioma" } ] }
+{  "cid": 586,  "name": "Jeannine Donnerberg",  "interests": [  ],  "children": [ {  "name": "Mike Donnerberg" } ] }
+{  "cid": 494,  "name": "Delma Deever",  "age": 84,  "address": {  "number": 5044,  "street": "7th St.",  "city": "Seattle" },  "interests": [ "Computers", "Basketball", "Squash" ],  "children": [  ] }
+{  "cid": 922,  "name": "Shanice Lingle",  "age": 26,  "address": {  "number": 4753,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Sandie Lingle",  "age": 12 }, {  "name": "Nia Lingle",  "age": 13 }, {  "name": "Marilyn Lingle",  "age": 15 } ] }
+{  "cid": 706,  "name": "Miquel Caesar",  "age": 16,  "address": {  "number": 2176,  "street": "Park St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Shaniqua Caesar" }, {  "name": "Ellis Caesar" }, {  "name": "Bruna Caesar" }, {  "name": "Kayleen Caesar" } ] }
+{  "cid": 425,  "name": "Hellen Sutton",  "interests": [ "Books", "Coffee", "Basketball", "Squash" ],  "children": [ {  "name": "Nancy Sutton" } ] }
+{  "cid": 313,  "name": "Lasandra Raigosa",  "interests": [ "Walking", "Walking" ],  "children": [ {  "name": "Lanelle Raigosa" } ] }
+{  "cid": 382,  "name": "Cecily Sopata",  "interests": [ "Base Jumping", "Fishing", "Skiing", "Squash" ],  "children": [ {  "name": "Shonna Sopata" }, {  "name": "Stacy Sopata" } ] }
+{  "cid": 975,  "name": "Gary Whitemore",  "interests": [  ],  "children": [  ] }
+{  "cid": 553,  "name": "Mina Ciminera",  "interests": [ "Base Jumping", "Databases" ],  "children": [ {  "name": "Cornelius Ciminera" }, {  "name": "Rozanne Ciminera" }, {  "name": "Byron Ciminera" } ] }
+{  "cid": 525,  "name": "Miquel Hodnefield",  "age": 12,  "address": {  "number": 4784,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Darnell Hodnefield" }, {  "name": "Particia Hodnefield" } ] }
+{  "cid": 472,  "name": "Kelley Mischler",  "age": 38,  "address": {  "number": 7988,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Movies", "Cooking", "Skiing" ],  "children": [ {  "name": "Keila Mischler",  "age": 19 }, {  "name": "Evie Mischler",  "age": 15 } ] }
+{  "cid": 759,  "name": "Alaina Dadds",  "interests": [  ],  "children": [ {  "name": "Athena Dadds",  "age": 36 }, {  "name": "Denis Dadds" }, {  "name": "Nathanial Dadds",  "age": 42 }, {  "name": "Molly Dadds" } ] }
+{  "cid": 573,  "name": "Tyree Ketcher",  "interests": [ "Computers", "Walking" ],  "children": [ {  "name": "Aleisha Ketcher" }, {  "name": "Vonda Ketcher" }, {  "name": "Cyndy Ketcher",  "age": 13 }, {  "name": "Chassidy Ketcher",  "age": 30 } ] }
+{  "cid": 639,  "name": "Zena Seehusen",  "age": 24,  "address": {  "number": 6303,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Cooking", "Movies", "Music" ],  "children": [ {  "name": "Hester Seehusen" }, {  "name": "Coreen Seehusen",  "age": 12 } ] }
+{  "cid": 546,  "name": "Shawanna Lontz",  "interests": [ "Base Jumping", "Basketball", "Music", "Basketball" ],  "children": [ {  "name": "Stuart Lontz",  "age": 57 }, {  "name": "Elizbeth Lontz" }, {  "name": "Zulema Lontz",  "age": 45 }, {  "name": "Brett Lontz" } ] }
+{  "cid": 710,  "name": "Arlen Horka",  "interests": [ "Movies", "Coffee", "Walking" ],  "children": [ {  "name": "Valencia Horka" }, {  "name": "Wesley Horka" } ] }
+{  "cid": 373,  "name": "Heather Seward",  "interests": [ "Basketball" ],  "children": [ {  "name": "Glinda Seward",  "age": 59 }, {  "name": "Maribeth Seward" }, {  "name": "Teofila Seward" }, {  "name": "Clemencia Seward",  "age": 38 } ] }
+{  "cid": 169,  "name": "Casandra Fierge",  "age": 55,  "address": {  "number": 175,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Cigars" ],  "children": [  ] }
+{  "cid": 422,  "name": "Annmarie Whitcher",  "interests": [ "Cigars" ],  "children": [ {  "name": "Honey Whitcher" }, {  "name": "Dan Whitcher",  "age": 22 } ] }
+{  "cid": 377,  "name": "Zona Klint",  "age": 22,  "address": {  "number": 6320,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Evie Klint" }, {  "name": "Sharyl Klint",  "age": 11 }, {  "name": "Joaquina Klint",  "age": 11 }, {  "name": "Doloris Klint",  "age": 11 } ] }
+{  "cid": 139,  "name": "Micheline Argenal",  "interests": [ "Bass", "Walking", "Movies" ],  "children": [ {  "name": "Joye Argenal",  "age": 51 }, {  "name": "Richard Argenal",  "age": 46 }, {  "name": "Sarah Argenal",  "age": 21 }, {  "name": "Jacinda Argenal",  "age": 21 } ] }
+{  "cid": 713,  "name": "Galina Retterbush",  "interests": [ "Bass", "Squash" ],  "children": [ {  "name": "Janene Retterbush" }, {  "name": "Toby Retterbush",  "age": 15 }, {  "name": "Renato Retterbush" }, {  "name": "Annice Retterbush",  "age": 22 } ] }
+{  "cid": 349,  "name": "Cristine Hila",  "interests": [ "Books" ],  "children": [ {  "name": "Nyla Hila",  "age": 51 } ] }
+{  "cid": 690,  "name": "Gertrudis Gaetz",  "interests": [ "Fishing", "Cigars", "Coffee", "Wine" ],  "children": [  ] }
+{  "cid": 899,  "name": "Ada Kamealoha",  "interests": [  ],  "children": [ {  "name": "Juliann Kamealoha" }, {  "name": "Ilana Kamealoha",  "age": 25 }, {  "name": "Herminia Kamealoha",  "age": 55 }, {  "name": "Carli Kamealoha" } ] }
+{  "cid": 581,  "name": "Leigha Finkenbinder",  "interests": [  ],  "children": [ {  "name": "Lorine Finkenbinder",  "age": 29 }, {  "name": "Stephanie Finkenbinder",  "age": 28 } ] }
+{  "cid": 232,  "name": "Joey Potes",  "interests": [ "Bass", "Bass", "Base Jumping" ],  "children": [ {  "name": "Bobby Potes" } ] }
+{  "cid": 664,  "name": "Myra Dier",  "age": 37,  "address": {  "number": 8703,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Wine", "Movies", "Puzzles", "Cooking" ],  "children": [  ] }
+{  "cid": 416,  "name": "Marcelo Salzar",  "age": 74,  "address": {  "number": 4091,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Skiing", "Base Jumping", "Music", "Running" ],  "children": [ {  "name": "Nickole Salzar" }, {  "name": "Rafael Salzar" }, {  "name": "Lois Salzar",  "age": 29 }, {  "name": "Deeanna Salzar" } ] }
+{  "cid": 492,  "name": "Gene Alcazar",  "age": 59,  "address": {  "number": 9650,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Computers" ],  "children": [ {  "name": "Olympia Alcazar" }, {  "name": "Mark Alcazar",  "age": 37 }, {  "name": "Danilo Alcazar" } ] }
+{  "cid": 773,  "name": "Leatrice Zysett",  "interests": [  ],  "children": [ {  "name": "Bee Zysett",  "age": 30 }, {  "name": "Russ Zysett",  "age": 11 }, {  "name": "Jeff Zysett",  "age": 39 }, {  "name": "Herman Zysett",  "age": 27 } ] }
+{  "cid": 0,  "name": "Antonia Streva",  "age": 39,  "address": {  "number": 872,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Bass", "Tennis", "Bass", "Cooking" ],  "children": [ {  "name": "Jonathan Streva",  "age": 25 }, {  "name": "Gricelda Streva",  "age": 24 } ] }
+{  "cid": 314,  "name": "Gwendolyn Abeb",  "age": 85,  "address": {  "number": 3977,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Basketball", "Music", "Squash", "Walking" ],  "children": [ {  "name": "Aurelia Abeb",  "age": 14 }, {  "name": "Young Abeb" }, {  "name": "Shay Abeb" }, {  "name": "Lavina Abeb",  "age": 15 } ] }
+{  "cid": 468,  "name": "Raeann Conry",  "age": 68,  "address": {  "number": 4312,  "street": "Cedar St.",  "city": "Seattle" },  "interests": [ "Squash" ],  "children": [ {  "name": "Ellena Conry",  "age": 36 }, {  "name": "Lynwood Conry",  "age": 13 }, {  "name": "Coreen Conry",  "age": 23 } ] }
+{  "cid": 231,  "name": "Arianne Wedlow",  "age": 68,  "address": {  "number": 9663,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Birdie Wedlow",  "age": 32 }, {  "name": "Pearle Wedlow",  "age": 13 }, {  "name": "Jordon Wedlow",  "age": 43 }, {  "name": "Katherin Wedlow",  "age": 18 } ] }
+{  "cid": 62,  "name": "Kiley Machnik",  "interests": [  ],  "children": [  ] }
+{  "cid": 97,  "name": "Mui Slosek",  "interests": [  ],  "children": [ {  "name": "Susanne Slosek",  "age": 29 }, {  "name": "Colleen Slosek" } ] }
+{  "cid": 357,  "name": "Dario Lobach",  "interests": [  ],  "children": [ {  "name": "Kendall Lobach",  "age": 37 } ] }
+{  "cid": 845,  "name": "Burt Earp",  "age": 21,  "address": {  "number": 7626,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Computers" ],  "children": [ {  "name": "Denny Earp" }, {  "name": "Blaine Earp" }, {  "name": "Wilson Earp",  "age": 10 }, {  "name": "Joan Earp" } ] }
+{  "cid": 835,  "name": "Raphael Marzili",  "interests": [ "Music" ],  "children": [ {  "name": "Angelic Marzili",  "age": 38 } ] }
+{  "cid": 811,  "name": "Marti Whitmyre",  "interests": [ "Music", "Walking" ],  "children": [  ] }
+{  "cid": 383,  "name": "Marty Castine",  "interests": [  ],  "children": [ {  "name": "Nakisha Castine",  "age": 40 }, {  "name": "Mina Castine" }, {  "name": "Katrice Castine",  "age": 56 }, {  "name": "Reuben Castine" } ] }
+{  "cid": 309,  "name": "Lise Baiz",  "age": 46,  "address": {  "number": 352,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Bass", "Squash" ],  "children": [ {  "name": "Alisa Baiz",  "age": 18 }, {  "name": "Elidia Baiz",  "age": 28 }, {  "name": "Ray Baiz",  "age": 19 } ] }
+{  "cid": 295,  "name": "Guillermina Florek",  "age": 61,  "address": {  "number": 3704,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Movies", "Books" ],  "children": [ {  "name": "Donnie Florek" }, {  "name": "Jeannetta Florek",  "age": 38 }, {  "name": "Leigha Florek" }, {  "name": "Zenobia Florek",  "age": 10 } ] }
+{  "cid": 273,  "name": "Corrinne Seaquist",  "age": 24,  "address": {  "number": 6712,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Puzzles", "Coffee", "Wine" ],  "children": [ {  "name": "Mignon Seaquist" }, {  "name": "Leo Seaquist" } ] }
+{  "cid": 718,  "name": "Tandy Trick",  "age": 18,  "address": {  "number": 1215,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Fishing", "Fishing" ],  "children": [ {  "name": "Edyth Trick" }, {  "name": "Jimmy Trick" }, {  "name": "Jacquline Trick" }, {  "name": "Tyler Trick" } ] }
+{  "cid": 751,  "name": "Lydia Iannelli",  "interests": [  ],  "children": [ {  "name": "Teri Iannelli",  "age": 36 } ] }
+{  "cid": 18,  "name": "Dewayne Ardan",  "age": 32,  "address": {  "number": 8229,  "street": "Hill St.",  "city": "San Jose" },  "interests": [ "Wine", "Walking", "Bass" ],  "children": [ {  "name": "Wen Ardan" }, {  "name": "Sachiko Ardan",  "age": 11 }, {  "name": "Francis Ardan",  "age": 20 } ] }
+{  "cid": 113,  "name": "Alayna Daleske",  "age": 87,  "address": {  "number": 4739,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Hester Daleske" }, {  "name": "Magnolia Daleske" }, {  "name": "Bettye Daleske",  "age": 32 } ] }
+{  "cid": 910,  "name": "Everette Moe",  "interests": [  ],  "children": [ {  "name": "Berna Moe",  "age": 56 }, {  "name": "Harold Moe",  "age": 28 }, {  "name": "See Moe",  "age": 20 } ] }
+{  "cid": 355,  "name": "Elois Leckband",  "interests": [ "Skiing", "Wine" ],  "children": [  ] }
+{  "cid": 347,  "name": "Patrick Feighan",  "age": 34,  "address": {  "number": 7613,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Puzzles", "Books" ],  "children": [ {  "name": "Madaline Feighan" } ] }
+{  "cid": 213,  "name": "Micheal Evoy",  "age": 68,  "address": {  "number": 1219,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Skiing", "Computers", "Books", "Puzzles" ],  "children": [ {  "name": "Socorro Evoy" }, {  "name": "Gertude Evoy",  "age": 36 }, {  "name": "Araceli Evoy" }, {  "name": "Yasmin Evoy" } ] }
+{  "cid": 699,  "name": "Lyda Golomb",  "age": 46,  "address": {  "number": 5049,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Fishing", "Basketball" ],  "children": [ {  "name": "Shonta Golomb" }, {  "name": "Lynwood Golomb",  "age": 26 }, {  "name": "Leonila Golomb",  "age": 30 }, {  "name": "Alejandrina Golomb" } ] }
+{  "cid": 961,  "name": "Mirian Herpolsheimer",  "interests": [ "Music", "Fishing", "Computers" ],  "children": [ {  "name": "Larissa Herpolsheimer",  "age": 41 }, {  "name": "Markus Herpolsheimer" }, {  "name": "Natacha Herpolsheimer" } ] }
+{  "cid": 189,  "name": "Shyla Saathoff",  "age": 85,  "address": {  "number": 9679,  "street": "Main St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Johanne Saathoff",  "age": 61 }, {  "name": "Janett Saathoff" } ] }
+{  "cid": 190,  "name": "Kristel Axelson",  "interests": [ "Movies", "Books" ],  "children": [ {  "name": "Deja Axelson" } ] }
+{  "cid": 419,  "name": "Hector Brisbone",  "interests": [ "Databases", "Books", "Walking", "Databases" ],  "children": [ {  "name": "Frederick Brisbone",  "age": 17 } ] }
+{  "cid": 563,  "name": "Deirdre Landero",  "interests": [ "Books", "Fishing", "Video Games" ],  "children": [ {  "name": "Norman Landero",  "age": 59 }, {  "name": "Jennine Landero",  "age": 45 }, {  "name": "Rutha Landero",  "age": 19 }, {  "name": "Jackie Landero",  "age": 29 } ] }
+{  "cid": 593,  "name": "Danial Pittillo",  "age": 87,  "address": {  "number": 815,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Tennis", "Base Jumping" ],  "children": [ {  "name": "Neva Pittillo",  "age": 28 }, {  "name": "Brooks Pittillo" }, {  "name": "Randell Pittillo",  "age": 52 }, {  "name": "Allyson Pittillo",  "age": 51 } ] }
+{  "cid": 37,  "name": "Eliana Vient",  "age": 89,  "address": {  "number": 4882,  "street": "View St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Dario Vient",  "age": 43 } ] }
+{  "cid": 703,  "name": "Susanne Pettey",  "interests": [ "Squash", "Basketball", "Skiing" ],  "children": [ {  "name": "Nancey Pettey",  "age": 35 }, {  "name": "Lawana Pettey" }, {  "name": "Percy Pettey",  "age": 25 } ] }
+{  "cid": 765,  "name": "Mila Barman",  "interests": [ "Coffee", "Puzzles", "Bass", "Wine" ],  "children": [ {  "name": "Lucienne Barman" }, {  "name": "Marina Barman" } ] }
+{  "cid": 432,  "name": "Judi Vinet",  "age": 85,  "address": {  "number": 7304,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Wine" ],  "children": [ {  "name": "Golden Vinet",  "age": 20 }, {  "name": "Maragret Vinet" }, {  "name": "Keshia Vinet",  "age": 10 }, {  "name": "Gary Vinet",  "age": 73 } ] }
+{  "cid": 61,  "name": "Linsey Mose",  "age": 17,  "address": {  "number": 9198,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Tilda Mose" }, {  "name": "Lillie Mose" }, {  "name": "Robyn Mose" } ] }
+{  "cid": 924,  "name": "Kathleen Lash",  "interests": [  ],  "children": [ {  "name": "Clementina Lash",  "age": 58 }, {  "name": "Zula Lash" }, {  "name": "Mellissa Lash",  "age": 54 } ] }
+{  "cid": 820,  "name": "Lacy Caudill",  "age": 22,  "address": {  "number": 8679,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Wine" ],  "children": [ {  "name": "Sybil Caudill" } ] }
+{  "cid": 590,  "name": "Joye Burton",  "interests": [ "Bass", "Base Jumping" ],  "children": [ {  "name": "Noemi Burton",  "age": 19 }, {  "name": "Hulda Burton" }, {  "name": "Cleotilde Burton" }, {  "name": "Dara Burton" } ] }
+{  "cid": 969,  "name": "Laurinda Gnerre",  "age": 42,  "address": {  "number": 2284,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Walking", "Bass", "Fishing", "Video Games" ],  "children": [ {  "name": "Veronica Gnerre" } ] }
+{  "cid": 681,  "name": "Iliana Nagele",  "interests": [ "Movies", "Running" ],  "children": [ {  "name": "Sunny Nagele",  "age": 55 }, {  "name": "Waltraud Nagele",  "age": 39 }, {  "name": "Darron Nagele" } ] }
+{  "cid": 946,  "name": "Taylor Parrigan",  "interests": [ "Music" ],  "children": [ {  "name": "Salome Parrigan",  "age": 50 }, {  "name": "Gary Parrigan",  "age": 25 }, {  "name": "Harold Parrigan" } ] }
+{  "cid": 170,  "name": "Dana Lese",  "age": 38,  "address": {  "number": 575,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Walking", "Coffee" ],  "children": [ {  "name": "Yasmine Lese",  "age": 24 }, {  "name": "Ezekiel Lese",  "age": 20 }, {  "name": "Ammie Lese",  "age": 27 }, {  "name": "Robert Lese",  "age": 15 } ] }
+{  "cid": 435,  "name": "Britni Kazemi",  "age": 69,  "address": {  "number": 7868,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Databases", "Music", "Wine" ],  "children": [  ] }
+{  "cid": 900,  "name": "Rose Mascetti",  "age": 73,  "address": {  "number": 5308,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Databases", "Coffee", "Computers", "Books" ],  "children": [  ] }
+{  "cid": 17,  "name": "Ingeborg Monkhouse",  "interests": [ "Base Jumping", "Cigars", "Movies" ],  "children": [  ] }
+{  "cid": 887,  "name": "Jermaine Folz",  "age": 35,  "address": {  "number": 8487,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Computers", "Puzzles", "Cooking" ],  "children": [ {  "name": "Sharice Folz" } ] }
+{  "cid": 150,  "name": "Jesus Vanleeuwen",  "interests": [  ],  "children": [ {  "name": "Sueann Vanleeuwen",  "age": 47 }, {  "name": "Refugia Vanleeuwen" }, {  "name": "Taisha Vanleeuwen" }, {  "name": "Nathaniel Vanleeuwen" } ] }
+{  "cid": 267,  "name": "Renay Huddelston",  "age": 68,  "address": {  "number": 1939,  "street": "Washington St.",  "city": "Mountain View" },  "interests": [ "Wine", "Base Jumping" ],  "children": [ {  "name": "Colene Huddelston" } ] }
+{  "cid": 652,  "name": "Armida Moeuy",  "age": 34,  "address": {  "number": 8306,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Running" ],  "children": [ {  "name": "Sunshine Moeuy" }, {  "name": "Leta Moeuy",  "age": 19 } ] }
+{  "cid": 747,  "name": "Gil Dunnaway",  "age": 65,  "address": {  "number": 3022,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Running", "Squash" ],  "children": [ {  "name": "Laurice Dunnaway" } ] }
+{  "cid": 689,  "name": "Camila Cho",  "age": 70,  "address": {  "number": 7731,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Video Games", "Cigars" ],  "children": [ {  "name": "Myrtie Cho",  "age": 57 }, {  "name": "Merideth Cho",  "age": 45 }, {  "name": "Meta Cho",  "age": 20 } ] }
+{  "cid": 426,  "name": "Agripina Philley",  "age": 79,  "address": {  "number": 1533,  "street": "Main St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Georgianne Philley" }, {  "name": "Neville Philley" }, {  "name": "Brande Philley",  "age": 42 }, {  "name": "Tanisha Philley" } ] }
+{  "cid": 330,  "name": "Noma Tollefsen",  "interests": [  ],  "children": [ {  "name": "Melody Tollefsen",  "age": 45 }, {  "name": "Caridad Tollefsen",  "age": 15 } ] }
+{  "cid": 767,  "name": "Wendi Hoecker",  "interests": [  ],  "children": [  ] }
+{  "cid": 673,  "name": "Willard Matuszek",  "interests": [ "Running" ],  "children": [ {  "name": "Kyong Matuszek" }, {  "name": "Delena Matuszek" }, {  "name": "Toney Matuszek" }, {  "name": "Shayne Matuszek",  "age": 19 } ] }
+{  "cid": 948,  "name": "Thad Scialpi",  "age": 22,  "address": {  "number": 8731,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Base Jumping", "Tennis", "Wine" ],  "children": [ {  "name": "Harlan Scialpi",  "age": 10 }, {  "name": "Lucile Scialpi",  "age": 11 }, {  "name": "Audria Scialpi" } ] }
+{  "cid": 684,  "name": "Elmo Ballenger",  "age": 69,  "address": {  "number": 2657,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Wine" ],  "children": [ {  "name": "Sheena Ballenger",  "age": 53 }, {  "name": "Abby Ballenger" }, {  "name": "Markus Ballenger" } ] }
+{  "cid": 311,  "name": "Ria Haflett",  "age": 14,  "address": {  "number": 9513,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Walking" ],  "children": [ {  "name": "Jimmie Haflett" }, {  "name": "Dario Haflett" }, {  "name": "Robbyn Haflett" } ] }
+{  "cid": 181,  "name": "Toni Sanghani",  "interests": [  ],  "children": [ {  "name": "Hollie Sanghani",  "age": 29 } ] }
+{  "cid": 600,  "name": "Cordell Sherburn",  "interests": [ "Squash", "Skiing", "Skiing" ],  "children": [ {  "name": "Shenna Sherburn",  "age": 22 }, {  "name": "Minna Sherburn",  "age": 10 }, {  "name": "Tari Sherburn" } ] }
+{  "cid": 753,  "name": "Maris Bannett",  "interests": [ "Fishing", "Cigars", "Running" ],  "children": [ {  "name": "Libbie Bannett",  "age": 11 }, {  "name": "Francina Bannett",  "age": 21 }, {  "name": "Tuyet Bannett" }, {  "name": "Zona Bannett",  "age": 32 } ] }
+{  "cid": 132,  "name": "Cindi Turntine",  "age": 64,  "address": {  "number": 9432,  "street": "Park St.",  "city": "Portland" },  "interests": [ "Computers", "Wine" ],  "children": [ {  "name": "Howard Turntine" } ] }
+{  "cid": 70,  "name": "Mellisa Lek",  "age": 62,  "address": {  "number": 4281,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Bass", "Running", "Databases" ],  "children": [  ] }
+{  "cid": 858,  "name": "Maricruz Dittberner",  "interests": [ "Tennis", "Wine", "Cigars", "Video Games" ],  "children": [  ] }
+{  "cid": 777,  "name": "Coralee Vaugh",  "age": 51,  "address": {  "number": 4130,  "street": "Hill St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Dean Vaugh",  "age": 31 }, {  "name": "Stanton Vaugh",  "age": 39 }, {  "name": "Marti Vaugh",  "age": 33 }, {  "name": "Eden Vaugh",  "age": 27 } ] }
+{  "cid": 2,  "name": "Elin Debell",  "age": 82,  "address": {  "number": 5649,  "street": "Hill St.",  "city": "Portland" },  "interests": [ "Bass", "Wine" ],  "children": [ {  "name": "Elvina Debell" }, {  "name": "Renaldo Debell",  "age": 51 }, {  "name": "Divina Debell",  "age": 57 } ] }
+{  "cid": 98,  "name": "Casimira Hilbrand",  "age": 72,  "address": {  "number": 9693,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Gudrun Hilbrand",  "age": 18 }, {  "name": "Dacia Hilbrand",  "age": 26 }, {  "name": "Kortney Hilbrand" }, {  "name": "Luci Hilbrand" } ] }
+{  "cid": 670,  "name": "Angelo Kellar",  "age": 22,  "address": {  "number": 3178,  "street": "View St.",  "city": "Seattle" },  "interests": [ "Wine", "Music", "Fishing" ],  "children": [ {  "name": "Zula Kellar" }, {  "name": "Brittaney Kellar",  "age": 10 }, {  "name": "Fredia Kellar" } ] }
+{  "cid": 981,  "name": "Lilliam Lopus",  "interests": [  ],  "children": [ {  "name": "Tracey Lopus" } ] }
+{  "cid": 669,  "name": "Royal Abke",  "age": 60,  "address": {  "number": 1675,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Leandra Abke",  "age": 25 }, {  "name": "Shawanna Abke" } ] }
+{  "cid": 146,  "name": "Glennis Vanruiten",  "age": 14,  "address": {  "number": 8272,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Squash", "Databases" ],  "children": [ {  "name": "Joanie Vanruiten" }, {  "name": "Long Vanruiten" }, {  "name": "Abdul Vanruiten" } ] }
+{  "cid": 431,  "name": "Estela Tolbent",  "age": 27,  "address": {  "number": 7186,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Databases" ],  "children": [ {  "name": "Joie Tolbent" }, {  "name": "Angila Tolbent" }, {  "name": "Anastasia Tolbent",  "age": 14 } ] }
+{  "cid": 199,  "name": "Rogelio Hannan",  "interests": [  ],  "children": [ {  "name": "Blanche Hannan" }, {  "name": "Elvira Hannan" }, {  "name": "Cinderella Hannan" } ] }
+{  "cid": 248,  "name": "Elsy Slack",  "interests": [ "Cooking", "Squash", "Cooking", "Coffee" ],  "children": [  ] }
+{  "cid": 143,  "name": "Katelynn Kanzler",  "age": 80,  "address": {  "number": 9453,  "street": "Washington St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Carl Kanzler" } ] }
+{  "cid": 487,  "name": "Zenia Virgilio",  "age": 46,  "address": {  "number": 584,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Walking", "Squash", "Wine" ],  "children": [ {  "name": "Quintin Virgilio" }, {  "name": "Edith Virgilio" }, {  "name": "Nicolle Virgilio",  "age": 33 } ] }
+{  "cid": 285,  "name": "Edgar Farlin",  "age": 75,  "address": {  "number": 3833,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Coffee", "Databases" ],  "children": [ {  "name": "Stefanie Farlin",  "age": 60 }, {  "name": "Catina Farlin" }, {  "name": "Lizzie Farlin" }, {  "name": "Beau Farlin" } ] }
+{  "cid": 161,  "name": "Lucia Tata",  "age": 85,  "address": {  "number": 8058,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Basketball", "Bass" ],  "children": [ {  "name": "Jenifer Tata",  "age": 70 }, {  "name": "Erna Tata" } ] }
+{  "cid": 226,  "name": "Debrah Deppert",  "age": 62,  "address": {  "number": 7699,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Coffee" ],  "children": [ {  "name": "Tonie Deppert",  "age": 25 }, {  "name": "Neil Deppert" } ] }
+{  "cid": 216,  "name": "Odilia Lampson",  "interests": [ "Wine", "Databases", "Basketball" ],  "children": [ {  "name": "Callie Lampson" } ] }
+{  "cid": 40,  "name": "Fidelia Connie",  "age": 81,  "address": {  "number": 2298,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Basketball", "Base Jumping", "Walking", "Skiing" ],  "children": [ {  "name": "Elfreda Connie",  "age": 43 }, {  "name": "Josephine Connie",  "age": 30 }, {  "name": "Lucas Connie" } ] }
+{  "cid": 209,  "name": "Donnette Kreb",  "interests": [ "Puzzles", "Cooking", "Tennis", "Tennis" ],  "children": [ {  "name": "Hobert Kreb" }, {  "name": "Ray Kreb" }, {  "name": "Carmel Kreb",  "age": 56 }, {  "name": "Lise Kreb" } ] }
+{  "cid": 766,  "name": "Tosha Loffredo",  "age": 64,  "address": {  "number": 5580,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Walking" ],  "children": [ {  "name": "Hellen Loffredo",  "age": 32 } ] }
+{  "cid": 656,  "name": "Rufus Peaden",  "interests": [  ],  "children": [ {  "name": "Nathanael Peaden",  "age": 57 }, {  "name": "Jamaal Peaden" } ] }
+{  "cid": 140,  "name": "Maryland Neas",  "interests": [  ],  "children": [ {  "name": "Brunilda Neas",  "age": 28 } ] }
+{  "cid": 711,  "name": "Agnes Andreas",  "interests": [ "Books" ],  "children": [ {  "name": "Fairy Andreas" }, {  "name": "Wilhemina Andreas" }, {  "name": "Parthenia Andreas",  "age": 53 }, {  "name": "Maye Andreas" } ] }
+{  "cid": 692,  "name": "Nida Picknell",  "age": 24,  "address": {  "number": 9053,  "street": "Park St.",  "city": "Mountain View" },  "interests": [ "Skiing", "Music", "Wine", "Base Jumping" ],  "children": [ {  "name": "Caroyln Picknell" }, {  "name": "Micheline Picknell",  "age": 10 } ] }
+{  "cid": 229,  "name": "Raymundo Meurin",  "interests": [ "Bass", "Basketball", "Databases" ],  "children": [ {  "name": "Mariela Meurin" } ] }
+{  "cid": 594,  "name": "Zenia Corban",  "interests": [ "Puzzles", "Computers", "Video Games", "Cigars" ],  "children": [ {  "name": "Arielle Corban" }, {  "name": "Arthur Corban",  "age": 15 }, {  "name": "Taneka Corban",  "age": 51 }, {  "name": "Claire Corban" } ] }
+{  "cid": 927,  "name": "Lillia Hartlein",  "age": 55,  "address": {  "number": 5856,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Base Jumping", "Coffee", "Cigars" ],  "children": [ {  "name": "Nicky Hartlein" }, {  "name": "Cassaundra Hartlein",  "age": 10 }, {  "name": "Micheline Hartlein",  "age": 26 }, {  "name": "Anton Hartlein",  "age": 32 } ] }
+{  "cid": 906,  "name": "Marlena Reichenberg",  "interests": [ "Tennis", "Bass", "Cigars", "Databases" ],  "children": [ {  "name": "Annemarie Reichenberg",  "age": 54 }, {  "name": "Sunshine Reichenberg" }, {  "name": "Dion Reichenberg",  "age": 49 }, {  "name": "Brenda Reichenberg",  "age": 43 } ] }
+{  "cid": 350,  "name": "Lashandra Noto",  "interests": [ "Movies", "Bass", "Coffee", "Squash" ],  "children": [ {  "name": "Lise Noto" }, {  "name": "Kimbra Noto",  "age": 36 }, {  "name": "Samual Noto" } ] }
+{  "cid": 102,  "name": "Melany Rotan",  "interests": [  ],  "children": [ {  "name": "Christiana Rotan",  "age": 21 }, {  "name": "Lavina Rotan" }, {  "name": "Billy Rotan" } ] }
+{  "cid": 804,  "name": "Joaquina Burlin",  "age": 77,  "address": {  "number": 5479,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Running", "Wine", "Running" ],  "children": [  ] }
+{  "cid": 241,  "name": "Lesha Ambrosia",  "age": 49,  "address": {  "number": 6133,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Base Jumping", "Running" ],  "children": [ {  "name": "Venice Ambrosia" } ] }
+{  "cid": 3,  "name": "Phung Wheetley",  "age": 12,  "address": {  "number": 5549,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Wine" ],  "children": [ {  "name": "Raelene Wheetley" }, {  "name": "Dudley Wheetley" } ] }
+{  "cid": 480,  "name": "Nigel Pitmon",  "interests": [ "Puzzles", "Books" ],  "children": [ {  "name": "Janene Pitmon" }, {  "name": "Louie Pitmon",  "age": 19 }, {  "name": "Genny Pitmon",  "age": 24 }, {  "name": "Robby Pitmon",  "age": 55 } ] }
+{  "cid": 795,  "name": "Sharilyn Branstad",  "interests": [ "Databases", "Music" ],  "children": [ {  "name": "Ashlee Branstad",  "age": 24 }, {  "name": "Bobbye Branstad",  "age": 26 }, {  "name": "Natalya Branstad" }, {  "name": "Edith Branstad" } ] }
+{  "cid": 548,  "name": "Elvia Duchesney",  "interests": [ "Basketball" ],  "children": [ {  "name": "Arcelia Duchesney",  "age": 22 } ] }
+{  "cid": 317,  "name": "Zona Caffarel",  "age": 52,  "address": {  "number": 9419,  "street": "Cedar St.",  "city": "Seattle" },  "interests": [ "Tennis", "Coffee" ],  "children": [ {  "name": "Cortez Caffarel" } ] }
+{  "cid": 77,  "name": "Chantal Parriera",  "age": 78,  "address": {  "number": 5967,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Squash", "Movies", "Coffee" ],  "children": [  ] }
+{  "cid": 970,  "name": "Pia Sudderth",  "interests": [ "Databases" ],  "children": [ {  "name": "Ernestina Sudderth",  "age": 15 }, {  "name": "Larue Sudderth",  "age": 46 }, {  "name": "Toshia Sudderth",  "age": 27 } ] }
+{  "cid": 68,  "name": "Chery Basini",  "interests": [ "Video Games" ],  "children": [  ] }
+{  "cid": 283,  "name": "Pilar Fritts",  "interests": [ "Tennis" ],  "children": [ {  "name": "Jeneva Fritts" }, {  "name": "Gail Fritts",  "age": 25 } ] }
+{  "cid": 568,  "name": "Marilou Veeder",  "age": 26,  "address": {  "number": 5722,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Coffee", "Databases", "Books", "Skiing" ],  "children": [  ] }
+{  "cid": 569,  "name": "Beata Diles",  "age": 88,  "address": {  "number": 2198,  "street": "Park St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Myrtice Diles",  "age": 46 }, {  "name": "Stella Diles" }, {  "name": "Rowena Diles",  "age": 26 } ] }
+{  "cid": 792,  "name": "Cassandra Servey",  "interests": [ "Databases", "Music", "Books", "Cigars" ],  "children": [  ] }
+{  "cid": 482,  "name": "Samantha Stonis",  "interests": [ "Databases" ],  "children": [  ] }
+{  "cid": 663,  "name": "Riley Noteboom",  "interests": [  ],  "children": [ {  "name": "Marvis Noteboom",  "age": 57 } ] }
+{  "cid": 951,  "name": "Janine Martorano",  "age": 65,  "address": {  "number": 6420,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Books", "Music" ],  "children": [ {  "name": "Idella Martorano" } ] }
+{  "cid": 29,  "name": "Ruthanne Tavana",  "interests": [ "Movies" ],  "children": [  ] }
+{  "cid": 610,  "name": "Elinor Notoma",  "age": 66,  "address": {  "number": 6763,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Coffee" ],  "children": [ {  "name": "Dennis Notoma" }, {  "name": "Carol Notoma",  "age": 21 } ] }
+{  "cid": 122,  "name": "Wei Perpall",  "age": 43,  "address": {  "number": 916,  "street": "Washington St.",  "city": "Los Angeles" },  "interests": [ "Bass" ],  "children": [ {  "name": "Mitchel Perpall",  "age": 11 }, {  "name": "Aliza Perpall" }, {  "name": "King Perpall" }, {  "name": "Santana Perpall",  "age": 22 } ] }
+{  "cid": 27,  "name": "Hollie Hyun",  "interests": [ "Skiing", "Walking" ],  "children": [ {  "name": "Morton Hyun" }, {  "name": "Farrah Hyun",  "age": 40 }, {  "name": "Ali Hyun" } ] }
+{  "cid": 356,  "name": "Pearlene Sakumoto",  "age": 22,  "address": {  "number": 5895,  "street": "7th St.",  "city": "San Jose" },  "interests": [ "Computers", "Bass", "Base Jumping", "Coffee" ],  "children": [  ] }
+{  "cid": 234,  "name": "Ilana Brothern",  "age": 36,  "address": {  "number": 4850,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Puzzles", "Walking", "Fishing" ],  "children": [ {  "name": "Shayne Brothern" }, {  "name": "Phillis Brothern" } ] }
+{  "cid": 506,  "name": "Jonna Kolbusz",  "interests": [  ],  "children": [ {  "name": "Debrah Kolbusz" }, {  "name": "Hugh Kolbusz" } ] }
+{  "cid": 315,  "name": "Kallie Eiselein",  "interests": [ "Computers", "Tennis" ],  "children": [  ] }
+{  "cid": 633,  "name": "Shalon Grauberger",  "age": 34,  "address": {  "number": 765,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Music", "Base Jumping", "Tennis" ],  "children": [ {  "name": "Kris Grauberger",  "age": 14 }, {  "name": "Stuart Grauberger",  "age": 12 }, {  "name": "Billy Grauberger" } ] }
+{  "cid": 291,  "name": "Svetlana Moone",  "interests": [ "Skiing", "Computers", "Running", "Walking" ],  "children": [ {  "name": "Emelina Moone" }, {  "name": "Candi Moone" } ] }
+{  "cid": 427,  "name": "Janay Presutti",  "interests": [ "Walking" ],  "children": [ {  "name": "Julietta Presutti" } ] }
+{  "cid": 196,  "name": "Darwin Seekell",  "interests": [ "Skiing" ],  "children": [ {  "name": "Kathryne Seekell" }, {  "name": "Marlon Seekell" }, {  "name": "Shiloh Seekell",  "age": 51 } ] }
+{  "cid": 220,  "name": "Soila Hannemann",  "interests": [ "Wine", "Puzzles", "Basketball" ],  "children": [ {  "name": "Piper Hannemann",  "age": 44 } ] }
+{  "cid": 508,  "name": "Tiffany Kimmey",  "age": 64,  "address": {  "number": 8625,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Bass", "Walking" ],  "children": [  ] }
+{  "cid": 252,  "name": "Almeda Charity",  "age": 19,  "address": {  "number": 5553,  "street": "View St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Rosia Charity" } ] }
+{  "cid": 21,  "name": "Gidget Galamay",  "age": 34,  "address": {  "number": 2854,  "street": "Washington St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Brunilda Galamay" }, {  "name": "Bethel Galamay" }, {  "name": "Devon Galamay",  "age": 17 } ] }
+{  "cid": 174,  "name": "Taneka Baldassare",  "age": 50,  "address": {  "number": 5787,  "street": "Park St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Junko Baldassare" }, {  "name": "Denisha Baldassare" }, {  "name": "Hermina Baldassare",  "age": 17 }, {  "name": "Lexie Baldassare" } ] }
+{  "cid": 550,  "name": "Aleisha Brehon",  "age": 61,  "address": {  "number": 7835,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Squash" ],  "children": [ {  "name": "Vito Brehon" }, {  "name": "Matthew Brehon",  "age": 32 } ] }
+{  "cid": 498,  "name": "Arleen Sultzer",  "interests": [ "Coffee", "Movies", "Skiing" ],  "children": [ {  "name": "Norine Sultzer",  "age": 29 } ] }
+{  "cid": 780,  "name": "Penny Poortinga",  "interests": [  ],  "children": [ {  "name": "Estella Poortinga" } ] }
+{  "cid": 613,  "name": "Shanelle Leader",  "interests": [ "Databases", "Base Jumping", "Wine", "Fishing" ],  "children": [ {  "name": "Florencia Leader" }, {  "name": "Herbert Leader",  "age": 11 }, {  "name": "Jeanna Leader" } ] }
+{  "cid": 503,  "name": "Phyliss Cassani",  "interests": [ "Squash", "Tennis" ],  "children": [ {  "name": "Rolando Cassani",  "age": 44 }, {  "name": "Rikki Cassani",  "age": 18 }, {  "name": "Monty Cassani",  "age": 40 } ] }
+{  "cid": 294,  "name": "Foster Salimi",  "age": 79,  "address": {  "number": 8439,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Pei Salimi" } ] }
+{  "cid": 708,  "name": "Elease Holtmann",  "age": 75,  "address": {  "number": 5295,  "street": "Washington St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Leonardo Holtmann" }, {  "name": "Katharine Holtmann" }, {  "name": "Chung Holtmann",  "age": 20 }, {  "name": "Teodoro Holtmann",  "age": 19 } ] }
+{  "cid": 547,  "name": "Daryl Dambra",  "interests": [  ],  "children": [ {  "name": "Jacquline Dambra" }, {  "name": "Seymour Dambra" } ] }
+{  "cid": 108,  "name": "Artie Boclair",  "age": 55,  "address": {  "number": 8555,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [ "Skiing", "Squash", "Skiing", "Fishing" ],  "children": [  ] }
+{  "cid": 509,  "name": "Alvaro Johnke",  "interests": [ "Computers" ],  "children": [ {  "name": "Allison Johnke" }, {  "name": "Ellan Johnke" } ] }
+{  "cid": 378,  "name": "Melany Matias",  "age": 10,  "address": {  "number": 8838,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Coffee", "Tennis", "Bass" ],  "children": [ {  "name": "Earnestine Matias" }, {  "name": "Lore Matias" } ] }
+{  "cid": 69,  "name": "Many Yeargain",  "interests": [ "Coffee" ],  "children": [ {  "name": "Brande Yeargain" }, {  "name": "Tawna Yeargain" }, {  "name": "Doris Yeargain" }, {  "name": "Valeria Yeargain",  "age": 51 } ] }
+{  "cid": 912,  "name": "Alessandra Kaskey",  "age": 52,  "address": {  "number": 6906,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Skiing", "Walking", "Basketball" ],  "children": [ {  "name": "Mack Kaskey" } ] }
+{  "cid": 201,  "name": "Tiny Hoysradt",  "interests": [  ],  "children": [ {  "name": "Simon Hoysradt",  "age": 24 } ] }
+{  "cid": 124,  "name": "Kelley Dressman",  "interests": [ "Squash", "Databases", "Fishing" ],  "children": [ {  "name": "Evie Dressman" }, {  "name": "Fredericka Dressman" }, {  "name": "Leigh Dressman" }, {  "name": "Luna Dressman",  "age": 29 } ] }
+{  "cid": 583,  "name": "Bev Yerena",  "interests": [ "Puzzles", "Wine" ],  "children": [ {  "name": "Larhonda Yerena",  "age": 45 }, {  "name": "Josefina Yerena" }, {  "name": "Sydney Yerena",  "age": 42 } ] }
+{  "cid": 960,  "name": "Lenore Limardi",  "interests": [ "Music" ],  "children": [ {  "name": "Kris Limardi",  "age": 12 } ] }
+{  "cid": 32,  "name": "Tia Berkley",  "age": 30,  "address": {  "number": 4507,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Base Jumping", "Music" ],  "children": [ {  "name": "Carmon Berkley" }, {  "name": "Kristina Berkley" }, {  "name": "Cristi Berkley",  "age": 19 } ] }
+{  "cid": 365,  "name": "Aiko Curra",  "interests": [ "Fishing", "Fishing", "Bass", "Cooking" ],  "children": [ {  "name": "Janelle Curra" } ] }
+{  "cid": 384,  "name": "Perla Giarrano",  "age": 88,  "address": {  "number": 4523,  "street": "Cedar St.",  "city": "Seattle" },  "interests": [ "Base Jumping", "Cooking", "Tennis", "Cigars" ],  "children": [ {  "name": "Melania Giarrano",  "age": 71 }, {  "name": "Evalyn Giarrano",  "age": 67 }, {  "name": "Kathrine Giarrano" }, {  "name": "Lizeth Giarrano" } ] }
+{  "cid": 242,  "name": "Jerold Shabot",  "interests": [ "Fishing", "Walking", "Walking", "Puzzles" ],  "children": [ {  "name": "Marie Shabot",  "age": 26 } ] }
+{  "cid": 941,  "name": "Jamey Jakobson",  "interests": [ "Books", "Cooking", "Video Games" ],  "children": [ {  "name": "Elmer Jakobson",  "age": 14 }, {  "name": "Minh Jakobson",  "age": 30 } ] }
+{  "cid": 218,  "name": "Clarinda Stagliano",  "age": 76,  "address": {  "number": 3258,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Video Games", "Cigars" ],  "children": [  ] }
+{  "cid": 701,  "name": "Ahmed Schnider",  "age": 61,  "address": {  "number": 2619,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Cooking", "Bass", "Movies", "Video Games" ],  "children": [ {  "name": "Marcel Schnider",  "age": 13 }, {  "name": "Micaela Schnider",  "age": 28 }, {  "name": "Roderick Schnider" } ] }
+{  "cid": 148,  "name": "Coy Dulay",  "age": 66,  "address": {  "number": 9793,  "street": "Hill St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Emile Dulay" }, {  "name": "Letitia Dulay",  "age": 38 } ] }
+{  "cid": 612,  "name": "Keneth Ganie",  "age": 57,  "address": {  "number": 7712,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Cigars", "Base Jumping" ],  "children": [ {  "name": "Connie Ganie" }, {  "name": "Kamala Ganie",  "age": 25 }, {  "name": "Beulah Ganie",  "age": 15 } ] }
+{  "cid": 514,  "name": "Raleigh Belling",  "age": 56,  "address": {  "number": 7408,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Running" ],  "children": [  ] }
+{  "cid": 145,  "name": "Carey Bousman",  "age": 61,  "address": {  "number": 16,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Lynda Bousman",  "age": 32 }, {  "name": "Evalyn Bousman",  "age": 17 } ] }
+{  "cid": 883,  "name": "Odilia Bugtong",  "interests": [  ],  "children": [ {  "name": "Mark Bugtong",  "age": 15 }, {  "name": "Paula Bugtong" }, {  "name": "Jenee Bugtong",  "age": 17 }, {  "name": "Lilian Bugtong",  "age": 44 } ] }
+{  "cid": 957,  "name": "Lucius Schurr",  "age": 75,  "address": {  "number": 3918,  "street": "Main St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Willetta Schurr",  "age": 22 }, {  "name": "Andre Schurr" }, {  "name": "Merrilee Schurr",  "age": 32 } ] }
+{  "cid": 444,  "name": "Demetra Sava",  "interests": [ "Music", "Fishing", "Databases", "Wine" ],  "children": [ {  "name": "Fidel Sava",  "age": 16 } ] }
+{  "cid": 543,  "name": "Pearl Nollette",  "interests": [ "Base Jumping", "Running" ],  "children": [  ] }
+{  "cid": 798,  "name": "Senaida Hickerson",  "age": 59,  "address": {  "number": 8248,  "street": "7th St.",  "city": "San Jose" },  "interests": [ "Bass", "Coffee", "Video Games", "Coffee" ],  "children": [ {  "name": "Long Hickerson",  "age": 17 }, {  "name": "Logan Hickerson",  "age": 43 }, {  "name": "Toi Hickerson",  "age": 12 } ] }
+{  "cid": 307,  "name": "Abraham Lanphear",  "age": 20,  "address": {  "number": 7552,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Video Games" ],  "children": [ {  "name": "Toccara Lanphear" }, {  "name": "Milly Lanphear" } ] }
+{  "cid": 882,  "name": "Erin Birdsall",  "interests": [ "Music", "Walking", "Basketball", "Base Jumping" ],  "children": [ {  "name": "Bibi Birdsall" }, {  "name": "Richard Birdsall",  "age": 49 }, {  "name": "Evelina Birdsall",  "age": 33 } ] }
+{  "cid": 278,  "name": "Deb Nicole",  "age": 59,  "address": {  "number": 9003,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Books", "Computers", "Walking", "Cooking" ],  "children": [ {  "name": "Len Nicole" } ] }
+{  "cid": 638,  "name": "Obdulia Dicosmo",  "age": 14,  "address": {  "number": 9237,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Base Jumping", "Music", "Video Games", "Video Games" ],  "children": [ {  "name": "Han Dicosmo" }, {  "name": "Yang Dicosmo" } ] }
+{  "cid": 523,  "name": "Johanne Huls",  "interests": [ "Books", "Bass" ],  "children": [ {  "name": "Melynda Huls" }, {  "name": "Vicky Huls",  "age": 16 }, {  "name": "Charlott Huls" } ] }
+{  "cid": 461,  "name": "Dessie Schnibbe",  "interests": [  ],  "children": [  ] }
+{  "cid": 618,  "name": "Janella Hurtt",  "interests": [ "Skiing", "Coffee", "Skiing" ],  "children": [ {  "name": "Lupe Hurtt",  "age": 17 }, {  "name": "Jae Hurtt",  "age": 14 }, {  "name": "Evan Hurtt",  "age": 45 } ] }
+{  "cid": 65,  "name": "Voncile Villaneuva",  "age": 46,  "address": {  "number": 9976,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Skiing", "Basketball", "Running", "Running" ],  "children": [ {  "name": "An Villaneuva",  "age": 12 } ] }
+{  "cid": 346,  "name": "Elden Choma",  "interests": [  ],  "children": [ {  "name": "Valorie Choma" }, {  "name": "Leslee Choma" } ] }
+{  "cid": 380,  "name": "Silva Purdue",  "age": 33,  "address": {  "number": 1759,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Music", "Squash" ],  "children": [ {  "name": "Marshall Purdue" }, {  "name": "Yuki Purdue" }, {  "name": "Val Purdue",  "age": 12 }, {  "name": "Dominica Purdue" } ] }
+{  "cid": 974,  "name": "Alexis Malcomson",  "interests": [ "Movies", "Books" ],  "children": [ {  "name": "Kerri Malcomson" } ] }
+{  "cid": 195,  "name": "Annetta Demille",  "age": 17,  "address": {  "number": 5722,  "street": "Park St.",  "city": "Portland" },  "interests": [ "Bass" ],  "children": [ {  "name": "Natacha Demille" }, {  "name": "Giuseppe Demille" }, {  "name": "Kami Demille" }, {  "name": "Jewell Demille" } ] }
+{  "cid": 860,  "name": "Isabelle Sept",  "age": 88,  "address": {  "number": 4382,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Puzzles", "Books" ],  "children": [  ] }
+{  "cid": 117,  "name": "Leana Grims",  "interests": [ "Coffee", "Base Jumping", "Fishing", "Running" ],  "children": [ {  "name": "Tiara Grims" } ] }
+{  "cid": 566,  "name": "Asley Grow",  "interests": [ "Coffee", "Books", "Tennis" ],  "children": [ {  "name": "Dale Grow" } ] }
+{  "cid": 632,  "name": "Keeley Goga",  "interests": [ "Books", "Base Jumping" ],  "children": [ {  "name": "Walter Goga",  "age": 39 }, {  "name": "Chaya Goga" }, {  "name": "Melodie Goga" }, {  "name": "Isidro Goga",  "age": 32 } ] }
+{  "cid": 687,  "name": "Adriene Glowinski",  "interests": [  ],  "children": [  ] }
+{  "cid": 649,  "name": "Anisha Sender",  "interests": [ "Tennis", "Databases", "Bass" ],  "children": [ {  "name": "Viva Sender",  "age": 40 }, {  "name": "Terica Sender" } ] }
+{  "cid": 653,  "name": "Robbie Rhump",  "interests": [ "Squash", "Computers" ],  "children": [ {  "name": "Alishia Rhump",  "age": 14 }, {  "name": "Lyndsay Rhump",  "age": 27 } ] }
+{  "cid": 908,  "name": "Ferdinand Auila",  "age": 82,  "address": {  "number": 1071,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Base Jumping", "Running", "Wine" ],  "children": [ {  "name": "Ai Auila",  "age": 69 }, {  "name": "Laurel Auila" } ] }
+{  "cid": 778,  "name": "Shellie Sario",  "interests": [ "Puzzles" ],  "children": [  ] }
+{  "cid": 137,  "name": "Camellia Pressman",  "age": 81,  "address": {  "number": 3947,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Movies", "Books", "Bass" ],  "children": [ {  "name": "Dwana Pressman" }, {  "name": "Johnathan Pressman" }, {  "name": "Kasey Pressman" }, {  "name": "Mitch Pressman" } ] }
+{  "cid": 443,  "name": "Kylee Kowalczyk",  "age": 47,  "address": {  "number": 1555,  "street": "Hill St.",  "city": "Portland" },  "interests": [ "Music", "Books", "Books", "Wine" ],  "children": [ {  "name": "Erwin Kowalczyk",  "age": 29 } ] }
+{  "cid": 110,  "name": "Karmen Milanesi",  "age": 67,  "address": {  "number": 6223,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Squash", "Squash" ],  "children": [ {  "name": "Emely Milanesi" }, {  "name": "Adam Milanesi" }, {  "name": "Gregg Milanesi" }, {  "name": "Sean Milanesi",  "age": 37 } ] }
+{  "cid": 709,  "name": "Jazmine Twiddy",  "interests": [ "Puzzles", "Computers", "Wine" ],  "children": [ {  "name": "Veronika Twiddy",  "age": 21 } ] }
+{  "cid": 297,  "name": "Adeline Frierson",  "interests": [ "Coffee", "Computers", "Fishing" ],  "children": [ {  "name": "Marci Frierson" }, {  "name": "Rolanda Frierson" }, {  "name": "Del Frierson" } ] }
+{  "cid": 851,  "name": "Darrel Machia",  "age": 31,  "address": {  "number": 3290,  "street": "View St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Coy Machia",  "age": 13 }, {  "name": "Janean Machia",  "age": 13 }, {  "name": "Sandi Machia",  "age": 18 } ] }
+{  "cid": 512,  "name": "Paul Cobian",  "interests": [  ],  "children": [ {  "name": "Will Cobian",  "age": 30 }, {  "name": "Conrad Cobian",  "age": 35 }, {  "name": "Justin Cobian",  "age": 11 } ] }
+{  "cid": 420,  "name": "Coralie Regueira",  "interests": [ "Books", "Tennis" ],  "children": [ {  "name": "Latoyia Regueira",  "age": 31 }, {  "name": "Obdulia Regueira",  "age": 12 }, {  "name": "Herlinda Regueira" } ] }
+{  "cid": 926,  "name": "Krishna Barkdull",  "age": 31,  "address": {  "number": 2640,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [ "Cigars", "Skiing", "Video Games", "Coffee" ],  "children": [ {  "name": "Nilsa Barkdull" }, {  "name": "Denver Barkdull",  "age": 10 }, {  "name": "Jenell Barkdull",  "age": 15 } ] }
+{  "cid": 885,  "name": "Les Legere",  "age": 87,  "address": {  "number": 3998,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Bass", "Tennis", "Fishing" ],  "children": [ {  "name": "Concetta Legere",  "age": 45 }, {  "name": "Tamica Legere" }, {  "name": "Aurora Legere" } ] }
+{  "cid": 155,  "name": "Aubrey Kleve",  "age": 24,  "address": {  "number": 809,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Coffee", "Bass", "Bass", "Fishing" ],  "children": [  ] }
+{  "cid": 303,  "name": "Michel Bayird",  "age": 37,  "address": {  "number": 7939,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Shan Bayird",  "age": 12 } ] }
+{  "cid": 264,  "name": "Leon Yoshizawa",  "age": 81,  "address": {  "number": 608,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Running", "Books", "Running" ],  "children": [ {  "name": "Carmela Yoshizawa",  "age": 34 } ] }
+{  "cid": 500,  "name": "Tierra Bjorklund",  "interests": [ "Puzzles", "Skiing" ],  "children": [ {  "name": "Avelina Bjorklund",  "age": 54 }, {  "name": "Mallory Bjorklund" } ] }
+{  "cid": 299,  "name": "Jacob Wainman",  "age": 76,  "address": {  "number": 4551,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Base Jumping", "Wine", "Coffee" ],  "children": [ {  "name": "Abram Wainman",  "age": 28 }, {  "name": "Ramonita Wainman",  "age": 18 }, {  "name": "Sheryll Wainman" } ] }
+{  "cid": 822,  "name": "Shane Deleonardo",  "interests": [ "Skiing", "Books", "Fishing", "Puzzles" ],  "children": [  ] }
+{  "cid": 636,  "name": "Babara Shore",  "age": 83,  "address": {  "number": 9452,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Movies", "Tennis" ],  "children": [ {  "name": "Candy Shore",  "age": 58 }, {  "name": "Nanci Shore" }, {  "name": "Asia Shore" } ] }
+{  "cid": 423,  "name": "Elayne Twichell",  "interests": [ "Video Games", "Video Games", "Fishing", "Databases" ],  "children": [ {  "name": "Rickie Twichell",  "age": 27 }, {  "name": "Leonor Twichell" }, {  "name": "Shon Twichell",  "age": 39 } ] }
+{  "cid": 361,  "name": "Angela Lacki",  "age": 35,  "address": {  "number": 9710,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Skiing" ],  "children": [  ] }
+{  "cid": 596,  "name": "Juliane Maddy",  "interests": [ "Coffee", "Computers", "Walking", "Basketball" ],  "children": [ {  "name": "Joannie Maddy" }, {  "name": "Penny Maddy",  "age": 35 }, {  "name": "Joette Maddy",  "age": 35 }, {  "name": "Karla Maddy",  "age": 54 } ] }
+{  "cid": 894,  "name": "Reginald Julien",  "age": 16,  "address": {  "number": 1107,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Databases", "Wine" ],  "children": [ {  "name": "Arthur Julien" }, {  "name": "Evia Julien" } ] }
+{  "cid": 821,  "name": "Carole Edlund",  "age": 76,  "address": {  "number": 4008,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Computers", "Cooking", "Running", "Basketball" ],  "children": [ {  "name": "Garfield Edlund",  "age": 54 }, {  "name": "Brooks Edlund" }, {  "name": "Gertrudis Edlund" }, {  "name": "Tabitha Edlund",  "age": 58 } ] }
+{  "cid": 228,  "name": "Donnette Brumbley",  "interests": [ "Databases", "Music" ],  "children": [ {  "name": "Madlyn Brumbley" }, {  "name": "Apolonia Brumbley",  "age": 13 }, {  "name": "Stephine Brumbley" }, {  "name": "Zelma Brumbley",  "age": 51 } ] }
+{  "cid": 79,  "name": "Alyce Schoenle",  "age": 57,  "address": {  "number": 1345,  "street": "Main St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Stewart Schoenle",  "age": 16 }, {  "name": "Bruce Schoenle",  "age": 44 } ] }
+{  "cid": 133,  "name": "Carey Smitty",  "interests": [ "Books", "Bass", "Video Games", "Wine" ],  "children": [ {  "name": "Cyrstal Smitty",  "age": 31 } ] }
+{  "cid": 452,  "name": "Casie Marasigan",  "interests": [ "Walking", "Computers" ],  "children": [ {  "name": "Connie Marasigan" }, {  "name": "Kimberlie Marasigan" } ] }
+{  "cid": 80,  "name": "Dominique Gulbransen",  "interests": [ "Base Jumping", "Databases", "Movies", "Coffee" ],  "children": [ {  "name": "Elizabeth Gulbransen",  "age": 44 }, {  "name": "Lesley Gulbransen",  "age": 14 } ] }
+{  "cid": 391,  "name": "Lynn Gregory",  "age": 51,  "address": {  "number": 1249,  "street": "Hill St.",  "city": "San Jose" },  "interests": [  ],  "children": [ {  "name": "Jeannine Gregory" }, {  "name": "Jaymie Gregory" }, {  "name": "Lorrine Gregory",  "age": 37 } ] }
+{  "cid": 853,  "name": "Denisse Peralto",  "age": 25,  "address": {  "number": 3931,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Tennis", "Walking", "Basketball" ],  "children": [ {  "name": "Asha Peralto",  "age": 14 }, {  "name": "Clark Peralto" }, {  "name": "Jessika Peralto" }, {  "name": "Nadene Peralto" } ] }
+{  "cid": 923,  "name": "Bobbi Ursino",  "interests": [ "Movies", "Books", "Walking" ],  "children": [ {  "name": "Shon Ursino" }, {  "name": "Lorean Ursino" } ] }
+{  "cid": 976,  "name": "Madalyn Nidiffer",  "age": 35,  "address": {  "number": 7635,  "street": "Main St.",  "city": "San Jose" },  "interests": [ "Coffee", "Wine", "Music" ],  "children": [ {  "name": "Tricia Nidiffer",  "age": 10 }, {  "name": "Kevin Nidiffer",  "age": 24 }, {  "name": "Elyse Nidiffer" } ] }
+{  "cid": 473,  "name": "Cordell Solas",  "interests": [ "Squash", "Music", "Bass", "Puzzles" ],  "children": [ {  "name": "Douglass Solas" }, {  "name": "Claribel Solas" }, {  "name": "Fred Solas" }, {  "name": "Ahmed Solas",  "age": 21 } ] }
+{  "cid": 661,  "name": "Lorita Kraut",  "age": 43,  "address": {  "number": 5017,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Tennis", "Movies", "Bass" ],  "children": [ {  "name": "Mirian Kraut" } ] }
+{  "cid": 411,  "name": "Cindi Pepin",  "interests": [  ],  "children": [ {  "name": "Fallon Pepin",  "age": 39 }, {  "name": "Armanda Pepin" }, {  "name": "Loriann Pepin" }, {  "name": "Bambi Pepin",  "age": 43 } ] }
+{  "cid": 952,  "name": "Brianne Norg",  "age": 62,  "address": {  "number": 8650,  "street": "Washington St.",  "city": "San Jose" },  "interests": [ "Tennis", "Movies", "Computers", "Basketball" ],  "children": [ {  "name": "Cherish Norg",  "age": 41 }, {  "name": "Frances Norg",  "age": 49 }, {  "name": "Irwin Norg" } ] }
+{  "cid": 863,  "name": "Caroll Jett",  "age": 70,  "address": {  "number": 8918,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Wine", "Cigars", "Cooking", "Wine" ],  "children": [ {  "name": "Heide Jett",  "age": 58 }, {  "name": "Bernarda Jett",  "age": 47 }, {  "name": "Milagros Jett",  "age": 34 } ] }
+{  "cid": 418,  "name": "Gavin Delpino",  "interests": [ "Basketball", "Skiing", "Wine", "Fishing" ],  "children": [ {  "name": "Gianna Delpino" }, {  "name": "Carmella Delpino",  "age": 55 } ] }
+{  "cid": 943,  "name": "Kathryne Blacock",  "age": 82,  "address": {  "number": 3510,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Running", "Bass", "Music" ],  "children": [  ] }
+{  "cid": 707,  "name": "Nicholle Heibult",  "age": 67,  "address": {  "number": 1264,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Movies", "Basketball", "Squash", "Skiing" ],  "children": [  ] }
+{  "cid": 379,  "name": "Penney Huslander",  "age": 58,  "address": {  "number": 6919,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Cooking", "Running" ],  "children": [ {  "name": "Magaret Huslander" }, {  "name": "Dodie Huslander",  "age": 14 } ] }
+{  "cid": 211,  "name": "Kristian Knepshield",  "interests": [  ],  "children": [  ] }
+{  "cid": 574,  "name": "Camellia Toxey",  "age": 52,  "address": {  "number": 5437,  "street": "Hill St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Deandrea Toxey" }, {  "name": "Danille Toxey" } ] }
+{  "cid": 597,  "name": "Clarine Eutsey",  "age": 39,  "address": {  "number": 9112,  "street": "7th St.",  "city": "Portland" },  "interests": [ "Video Games", "Cigars", "Walking" ],  "children": [  ] }
+{  "cid": 457,  "name": "Jenice Boger",  "interests": [ "Skiing", "Databases", "Running" ],  "children": [  ] }
+{  "cid": 782,  "name": "Shameka Haifa",  "age": 16,  "address": {  "number": 9555,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Cigars", "Computers", "Coffee", "Skiing" ],  "children": [ {  "name": "Dannette Haifa" } ] }
+{  "cid": 86,  "name": "Sofia Mongiovi",  "interests": [  ],  "children": [ {  "name": "Rosamaria Mongiovi",  "age": 25 } ] }
+{  "cid": 861,  "name": "Hugh Mcbrien",  "interests": [ "Skiing", "Cigars", "Cooking" ],  "children": [ {  "name": "Otha Mcbrien",  "age": 38 } ] }
+{  "cid": 989,  "name": "Loyce Ferryman",  "age": 21,  "address": {  "number": 8937,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Puzzles", "Tennis", "Databases", "Base Jumping" ],  "children": [ {  "name": "Vada Ferryman" }, {  "name": "Reyes Ferryman" } ] }
+{  "cid": 901,  "name": "Riva Ziko",  "interests": [ "Running", "Tennis", "Video Games" ],  "children": [ {  "name": "Leandra Ziko",  "age": 49 }, {  "name": "Torrie Ziko" } ] }
+{  "cid": 433,  "name": "Caleb Merrbach",  "interests": [  ],  "children": [ {  "name": "Amado Merrbach",  "age": 45 } ] }
+{  "cid": 793,  "name": "Shondra Gollman",  "interests": [ "Skiing" ],  "children": [ {  "name": "Paul Gollman",  "age": 30 }, {  "name": "Katherina Gollman",  "age": 53 } ] }
+{  "cid": 263,  "name": "Mellisa Machalek",  "interests": [ "Bass", "Coffee", "Skiing" ],  "children": [  ] }
+{  "cid": 239,  "name": "Celsa Fondow",  "interests": [ "Base Jumping", "Computers", "Cooking", "Wine" ],  "children": [  ] }
+{  "cid": 852,  "name": "Terrell Ramsay",  "interests": [  ],  "children": [  ] }
+{  "cid": 918,  "name": "Melia Caparelli",  "age": 22,  "address": {  "number": 16,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles", "Fishing", "Coffee", "Music" ],  "children": [  ] }
+{  "cid": 534,  "name": "Bridgett Ebel",  "interests": [ "Cigars" ],  "children": [  ] }
+{  "cid": 744,  "name": "Crysta Christen",  "age": 57,  "address": {  "number": 439,  "street": "Hill St.",  "city": "Portland" },  "interests": [ "Basketball", "Squash", "Base Jumping" ],  "children": [  ] }
+{  "cid": 815,  "name": "Leigha Bires",  "age": 11,  "address": {  "number": 7263,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Running" ],  "children": [ {  "name": "Val Bires" } ] }
+{  "cid": 243,  "name": "Love Hoftiezer",  "age": 88,  "address": {  "number": 2491,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Cigars", "Coffee", "Books" ],  "children": [ {  "name": "Kellee Hoftiezer",  "age": 77 } ] }
+{  "cid": 561,  "name": "Renetta Cudworth",  "interests": [ "Skiing", "Basketball" ],  "children": [  ] }
+{  "cid": 429,  "name": "Eladia Scannell",  "age": 20,  "address": {  "number": 5036,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Skiing", "Music", "Movies" ],  "children": [  ] }
+{  "cid": 439,  "name": "Lillia Villnave",  "age": 34,  "address": {  "number": 9212,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Otis Villnave" } ] }
+{  "cid": 293,  "name": "Terresa Hofstetter",  "age": 15,  "address": {  "number": 3338,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Computers", "Running", "Cigars", "Fishing" ],  "children": [ {  "name": "Hubert Hofstetter" }, {  "name": "Jolie Hofstetter" } ] }
+{  "cid": 734,  "name": "Lera Korn",  "interests": [ "Tennis", "Puzzles", "Cigars" ],  "children": [ {  "name": "Criselda Korn",  "age": 37 } ] }
+{  "cid": 914,  "name": "Hunter Flournoy",  "interests": [ "Cooking", "Squash" ],  "children": [ {  "name": "Christopher Flournoy",  "age": 59 }, {  "name": "Earnestine Flournoy" } ] }
+{  "cid": 648,  "name": "Isaac Eagen",  "interests": [ "Fishing", "Cooking", "Basketball", "Books" ],  "children": [ {  "name": "Onita Eagen" }, {  "name": "Anjanette Eagen" } ] }
+{  "cid": 843,  "name": "Lenny Acerno",  "age": 64,  "address": {  "number": 7656,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Base Jumping", "Squash" ],  "children": [  ] }
+{  "cid": 100,  "name": "Taisha Wills",  "interests": [ "Base Jumping", "Music", "Skiing", "Databases" ],  "children": [  ] }
+{  "cid": 6,  "name": "Cris Kager",  "age": 70,  "address": {  "number": 8402,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Walking" ],  "children": [ {  "name": "Carmelo Kager",  "age": 34 }, {  "name": "Faustina Kager" } ] }
+{  "cid": 571,  "name": "Lenita Tentler",  "interests": [ "Running", "Fishing" ],  "children": [ {  "name": "Damian Tentler",  "age": 16 }, {  "name": "Camellia Tentler" }, {  "name": "Vern Tentler",  "age": 15 } ] }
+{  "cid": 324,  "name": "Wendolyn Centorino",  "interests": [  ],  "children": [  ] }
+{  "cid": 592,  "name": "Rachelle Spare",  "age": 13,  "address": {  "number": 8088,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Squash", "Puzzles" ],  "children": [ {  "name": "Theo Spare" }, {  "name": "Shizue Spare" } ] }
+{  "cid": 752,  "name": "Maria Lebovic",  "interests": [ "Bass" ],  "children": [ {  "name": "Thi Lebovic" }, {  "name": "Rosamaria Lebovic",  "age": 23 }, {  "name": "Brinda Lebovic",  "age": 39 } ] }
+{  "cid": 591,  "name": "Matthew Tenhaeff",  "interests": [ "Databases", "Video Games" ],  "children": [ {  "name": "Jan Tenhaeff",  "age": 25 }, {  "name": "Nana Tenhaeff" }, {  "name": "Laticia Tenhaeff" }, {  "name": "Ara Tenhaeff",  "age": 44 } ] }
+{  "cid": 404,  "name": "Harriette Abo",  "interests": [ "Walking", "Running" ],  "children": [  ] }
+{  "cid": 621,  "name": "Theresa Satterthwaite",  "age": 16,  "address": {  "number": 3249,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Wine", "Skiing", "Wine", "Fishing" ],  "children": [ {  "name": "Rickie Satterthwaite" }, {  "name": "Rina Satterthwaite" } ] }
+{  "cid": 215,  "name": "Ashton Schadegg",  "interests": [ "Databases", "Music" ],  "children": [ {  "name": "Ciara Schadegg" }, {  "name": "Karisa Schadegg",  "age": 11 }, {  "name": "Hayden Schadegg",  "age": 44 } ] }
+{  "cid": 640,  "name": "Willy Bielak",  "interests": [ "Squash" ],  "children": [  ] }
+{  "cid": 486,  "name": "Willa Patman",  "interests": [  ],  "children": [ {  "name": "Ross Patman",  "age": 42 }, {  "name": "Erin Patman" }, {  "name": "Vannessa Patman",  "age": 11 }, {  "name": "Hilaria Patman",  "age": 28 } ] }
+{  "cid": 389,  "name": "Loraine Morfee",  "age": 72,  "address": {  "number": 2945,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Wine", "Walking" ],  "children": [ {  "name": "Berry Morfee",  "age": 30 } ] }
+{  "cid": 339,  "name": "Sharonda Catalino",  "age": 15,  "address": {  "number": 7616,  "street": "Washington St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Lorine Catalino" } ] }
+{  "cid": 399,  "name": "Myra Millwee",  "interests": [ "Tennis", "Running", "Tennis" ],  "children": [ {  "name": "Gaye Millwee" } ] }
+{  "cid": 202,  "name": "Evangelina Poloskey",  "age": 46,  "address": {  "number": 8285,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [ "Wine", "Squash" ],  "children": [ {  "name": "Anthony Poloskey",  "age": 27 }, {  "name": "Olga Poloskey",  "age": 10 }, {  "name": "Carmon Poloskey",  "age": 13 }, {  "name": "Tanja Poloskey",  "age": 20 } ] }
+{  "cid": 631,  "name": "Brook Jenks",  "interests": [ "Wine" ],  "children": [ {  "name": "Eldon Jenks" }, {  "name": "Luann Jenks",  "age": 53 }, {  "name": "Aurora Jenks",  "age": 37 } ] }
+{  "cid": 549,  "name": "Kathrin Cruff",  "age": 63,  "address": {  "number": 9002,  "street": "Washington St.",  "city": "Sunnyvale" },  "interests": [ "Tennis", "Books" ],  "children": [ {  "name": "Candi Cruff",  "age": 49 }, {  "name": "Barry Cruff",  "age": 17 }, {  "name": "Shane Cruff",  "age": 18 }, {  "name": "Brendon Cruff" } ] }
+{  "cid": 179,  "name": "Antonette Bernice",  "interests": [  ],  "children": [ {  "name": "Solange Bernice" } ] }
+{  "cid": 74,  "name": "Lonnie Ercolani",  "age": 79,  "address": {  "number": 2655,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Music", "Coffee" ],  "children": [ {  "name": "Cassi Ercolani" } ] }
+{  "cid": 249,  "name": "Kiana Satiago",  "interests": [  ],  "children": [ {  "name": "Stacy Satiago" } ] }
+{  "cid": 114,  "name": "Stephine Capinpin",  "age": 78,  "address": {  "number": 5618,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles", "Basketball" ],  "children": [ {  "name": "Krystal Capinpin",  "age": 31 }, {  "name": "Angelic Capinpin",  "age": 45 } ] }
+{  "cid": 945,  "name": "Hildegard Dedinas",  "age": 70,  "address": {  "number": 3273,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Renato Dedinas",  "age": 35 } ] }
+{  "cid": 397,  "name": "Blake Kealy",  "age": 34,  "address": {  "number": 2156,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Wine", "Cigars" ],  "children": [ {  "name": "Lorenza Kealy" }, {  "name": "Beula Kealy",  "age": 15 }, {  "name": "Kristofer Kealy" }, {  "name": "Shayne Kealy" } ] }
+{  "cid": 915,  "name": "Eugene Okorududu",  "age": 62,  "address": {  "number": 8364,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Wine", "Skiing", "Cooking", "Movies" ],  "children": [ {  "name": "Renee Okorududu" }, {  "name": "Enid Okorududu" }, {  "name": "Tammy Okorududu" }, {  "name": "Shirlee Okorududu",  "age": 28 } ] }
+{  "cid": 646,  "name": "Pablo Catterton",  "interests": [ "Fishing", "Computers" ],  "children": [  ] }
+{  "cid": 359,  "name": "Sharika Vientos",  "age": 42,  "address": {  "number": 5981,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [ "Walking", "Bass", "Fishing", "Movies" ],  "children": [ {  "name": "Clifton Vientos",  "age": 21 }, {  "name": "Renae Vientos" }, {  "name": "Marcelo Vientos",  "age": 31 }, {  "name": "Jacalyn Vientos" } ] }
+{  "cid": 99,  "name": "Bernardina Thacher",  "age": 35,  "address": {  "number": 1582,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [ "Movies", "Fishing", "Fishing" ],  "children": [ {  "name": "Randee Thacher" }, {  "name": "China Thacher" } ] }
+{  "cid": 58,  "name": "Rosemarie Mattei",  "age": 80,  "address": {  "number": 1390,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Sonya Mattei",  "age": 52 }, {  "name": "Elenor Mattei" } ] }
+{  "cid": 931,  "name": "Octavia Koiner",  "interests": [  ],  "children": [ {  "name": "Ardath Koiner",  "age": 32 }, {  "name": "Milly Koiner" }, {  "name": "Arlinda Koiner" }, {  "name": "Debby Koiner" } ] }
+{  "cid": 779,  "name": "Vinita Bockskopf",  "interests": [ "Tennis", "Video Games" ],  "children": [  ] }
+{  "cid": 491,  "name": "Tobi Celani",  "age": 63,  "address": {  "number": 2200,  "street": "Main St.",  "city": "Mountain View" },  "interests": [ "Fishing", "Running", "Bass", "Fishing" ],  "children": [ {  "name": "Alana Celani" }, {  "name": "Lashaun Celani" }, {  "name": "Sirena Celani",  "age": 23 }, {  "name": "Tami Celani" } ] }
+{  "cid": 131,  "name": "Kourtney Whitesel",  "interests": [  ],  "children": [  ] }
+{  "cid": 828,  "name": "Marcelle Steinhour",  "interests": [ "Running", "Basketball", "Walking" ],  "children": [ {  "name": "Jimmie Steinhour",  "age": 13 }, {  "name": "Kirstie Steinhour",  "age": 19 } ] }
+{  "cid": 304,  "name": "Francine Reddin",  "age": 39,  "address": {  "number": 9392,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Music", "Base Jumping" ],  "children": [ {  "name": "Millicent Reddin" } ] }
+{  "cid": 545,  "name": "Dolores Ferer",  "interests": [ "Coffee", "Bass", "Tennis" ],  "children": [ {  "name": "Bridgette Ferer" } ] }
+{  "cid": 992,  "name": "Staci Alexandropoul",  "interests": [ "Databases", "Movies", "Tennis" ],  "children": [ {  "name": "Casimira Alexandropoul" }, {  "name": "Kena Alexandropoul",  "age": 54 }, {  "name": "Ellie Alexandropoul" }, {  "name": "Ambrose Alexandropoul" } ] }
+{  "cid": 112,  "name": "Dorie Lave",  "age": 10,  "address": {  "number": 2286,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Coffee" ],  "children": [ {  "name": "Grady Lave" }, {  "name": "Daysi Lave" } ] }
+{  "cid": 47,  "name": "Britni Haider",  "age": 86,  "address": {  "number": 9172,  "street": "Park St.",  "city": "Seattle" },  "interests": [ "Basketball", "Fishing", "Tennis", "Fishing" ],  "children": [ {  "name": "Vergie Haider" } ] }
+{  "cid": 725,  "name": "Sallie Calderon",  "interests": [  ],  "children": [  ] }
+{  "cid": 354,  "name": "Marian Munzell",  "age": 73,  "address": {  "number": 4504,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Fishing", "Puzzles" ],  "children": [  ] }
+{  "cid": 210,  "name": "Jillian Roadruck",  "interests": [ "Coffee", "Tennis" ],  "children": [ {  "name": "Marguerite Roadruck" }, {  "name": "Ilana Roadruck" }, {  "name": "Chantelle Roadruck",  "age": 19 }, {  "name": "Nikia Roadruck",  "age": 43 } ] }
+{  "cid": 736,  "name": "Desmond Branam",  "interests": [  ],  "children": [ {  "name": "Manuel Branam",  "age": 51 } ] }
+{  "cid": 682,  "name": "Krystle Weingartner",  "age": 87,  "address": {  "number": 5293,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Squash" ],  "children": [ {  "name": "Bryanna Weingartner",  "age": 19 }, {  "name": "Rubie Weingartner",  "age": 32 }, {  "name": "Raye Weingartner" } ] }
+{  "cid": 637,  "name": "George Beamer",  "age": 53,  "address": {  "number": 9464,  "street": "Park St.",  "city": "Mountain View" },  "interests": [ "Fishing", "Running", "Books", "Music" ],  "children": [ {  "name": "Mayra Beamer",  "age": 12 }, {  "name": "Bernadette Beamer",  "age": 39 }, {  "name": "Nicky Beamer" }, {  "name": "Cheree Beamer" } ] }
+{  "cid": 10,  "name": "Trent Liedy",  "age": 51,  "address": {  "number": 1758,  "street": "Oak St.",  "city": "San Jose" },  "interests": [  ],  "children": [  ] }
+{  "cid": 60,  "name": "Dorthey Gradowski",  "interests": [ "Tennis", "Tennis", "Databases", "Squash" ],  "children": [ {  "name": "Andera Gradowski",  "age": 15 }, {  "name": "Demetrice Gradowski",  "age": 13 } ] }
+{  "cid": 50,  "name": "Lise Gorelli",  "interests": [ "Books", "Wine", "Skiing", "Computers" ],  "children": [ {  "name": "Darleen Gorelli" }, {  "name": "Latia Gorelli" }, {  "name": "Page Gorelli" }, {  "name": "Columbus Gorelli" } ] }
+{  "cid": 185,  "name": "Abigail Zugg",  "age": 22,  "address": {  "number": 6676,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Computers", "Basketball", "Video Games", "Basketball" ],  "children": [ {  "name": "Peter Zugg",  "age": 10 }, {  "name": "Ariane Zugg" } ] }
+{  "cid": 630,  "name": "Darla Domenick",  "age": 14,  "address": {  "number": 3315,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Databases" ],  "children": [ {  "name": "Verda Domenick" } ] }
+{  "cid": 453,  "name": "Sherlyn Deadmond",  "interests": [ "Tennis", "Puzzles", "Base Jumping" ],  "children": [ {  "name": "Torrie Deadmond",  "age": 46 }, {  "name": "Cleotilde Deadmond",  "age": 55 }, {  "name": "Garry Deadmond",  "age": 34 }, {  "name": "Valrie Deadmond" } ] }
+{  "cid": 785,  "name": "Gabriel Breidel",  "age": 32,  "address": {  "number": 9288,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Cigars", "Bass" ],  "children": [ {  "name": "Bernie Breidel" } ] }
+{  "cid": 540,  "name": "Bryanna Herling",  "age": 67,  "address": {  "number": 7682,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Cyrstal Herling",  "age": 50 }, {  "name": "Vallie Herling",  "age": 54 }, {  "name": "Doris Herling" } ] }
+{  "cid": 659,  "name": "Daniel Groskreutz",  "interests": [ "Databases" ],  "children": [ {  "name": "Mariam Groskreutz",  "age": 21 }, {  "name": "Carlton Groskreutz" } ] }
+{  "cid": 85,  "name": "Fatimah Steltenpohl",  "age": 25,  "address": {  "number": 6175,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Genoveva Steltenpohl",  "age": 14 } ] }
+{  "cid": 755,  "name": "Bette Trentz",  "age": 57,  "address": {  "number": 2794,  "street": "Park St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Christa Trentz",  "age": 14 }, {  "name": "Jestine Trentz",  "age": 22 }, {  "name": "Shantel Trentz",  "age": 37 }, {  "name": "Jacklyn Trentz" } ] }
+{  "cid": 942,  "name": "Emerson Keblish",  "interests": [ "Tennis" ],  "children": [ {  "name": "Leonora Keblish" } ] }
+{  "cid": 290,  "name": "Kimberly Gullatte",  "age": 51,  "address": {  "number": 4130,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Running", "Squash", "Databases" ],  "children": [ {  "name": "Micheal Gullatte" }, {  "name": "Estrella Gullatte",  "age": 40 }, {  "name": "Corrine Gullatte" }, {  "name": "Ward Gullatte" } ] }
+{  "cid": 115,  "name": "Jason Oakden",  "age": 89,  "address": {  "number": 8182,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Music", "Basketball", "Movies" ],  "children": [ {  "name": "Johnson Oakden" }, {  "name": "Neva Oakden" }, {  "name": "Juliann Oakden" }, {  "name": "Elmer Oakden" } ] }
+{  "cid": 409,  "name": "Edwardo Brayton",  "age": 28,  "address": {  "number": 473,  "street": "7th St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Basketball", "Computers", "Fishing" ],  "children": [ {  "name": "Werner Brayton" } ] }
+{  "cid": 757,  "name": "Bertie Flemming",  "interests": [ "Tennis", "Music", "Running", "Cooking" ],  "children": [ {  "name": "Temeka Flemming",  "age": 46 }, {  "name": "Terrance Flemming" }, {  "name": "Jenette Flemming",  "age": 23 }, {  "name": "Debra Flemming" } ] }
+{  "cid": 236,  "name": "Muriel Laib",  "age": 25,  "address": {  "number": 4481,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Fishing", "Tennis" ],  "children": [ {  "name": "Jann Laib" }, {  "name": "Lila Laib",  "age": 10 }, {  "name": "Elyse Laib",  "age": 11 } ] }
+{  "cid": 262,  "name": "Diane Bowersmith",  "interests": [ "Basketball", "Movies", "Music", "Squash" ],  "children": [ {  "name": "Errol Bowersmith",  "age": 16 }, {  "name": "Lien Bowersmith",  "age": 10 } ] }
+{  "cid": 575,  "name": "Phyliss Mattes",  "age": 26,  "address": {  "number": 3956,  "street": "Washington St.",  "city": "Los Angeles" },  "interests": [ "Tennis", "Music", "Running", "Music" ],  "children": [  ] }
+{  "cid": 799,  "name": "Ronny Piefer",  "age": 45,  "address": {  "number": 7724,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Fishing" ],  "children": [ {  "name": "Chantal Piefer",  "age": 24 }, {  "name": "Tiffany Piefer" }, {  "name": "Farrah Piefer",  "age": 21 }, {  "name": "Dee Piefer" } ] }
+{  "cid": 676,  "name": "Ima Juart",  "age": 64,  "address": {  "number": 2498,  "street": "Cedar St.",  "city": "Portland" },  "interests": [ "Walking" ],  "children": [ {  "name": "Cortez Juart",  "age": 17 }, {  "name": "Guillermo Juart" }, {  "name": "Shelley Juart",  "age": 20 }, {  "name": "Daryl Juart" } ] }
+{  "cid": 781,  "name": "Christy Darcangelo",  "age": 42,  "address": {  "number": 2178,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Computers", "Fishing" ],  "children": [ {  "name": "Luis Darcangelo",  "age": 21 }, {  "name": "Omega Darcangelo",  "age": 26 }, {  "name": "Remedios Darcangelo",  "age": 28 }, {  "name": "Domenic Darcangelo",  "age": 21 } ] }
+{  "cid": 495,  "name": "Lashaun Gaud",  "interests": [ "Music", "Music", "Coffee", "Basketball" ],  "children": [ {  "name": "Elizabeth Gaud" }, {  "name": "Eloise Gaud" }, {  "name": "Dell Gaud" }, {  "name": "Lala Gaud" } ] }
+{  "cid": 886,  "name": "Jerry Defusco",  "interests": [ "Databases", "Puzzles", "Puzzles", "Basketball" ],  "children": [ {  "name": "Caroyln Defusco" }, {  "name": "Eilene Defusco" } ] }
+{  "cid": 119,  "name": "Chan Morreau",  "age": 22,  "address": {  "number": 1774,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Puzzles", "Squash" ],  "children": [ {  "name": "Arlette Morreau" } ] }
+{  "cid": 750,  "name": "Rosaura Gaul",  "interests": [ "Music", "Books", "Tennis" ],  "children": [ {  "name": "Letisha Gaul",  "age": 41 } ] }
+{  "cid": 770,  "name": "Merrill Tilson",  "interests": [ "Computers", "Skiing" ],  "children": [ {  "name": "Elna Tilson" } ] }
+{  "cid": 374,  "name": "Clair Quinn",  "interests": [ "Walking", "Books" ],  "children": [ {  "name": "Wesley Quinn",  "age": 17 }, {  "name": "Maren Quinn",  "age": 50 }, {  "name": "Ila Quinn",  "age": 43 }, {  "name": "Casie Quinn" } ] }
+{  "cid": 812,  "name": "Bee Godette",  "age": 26,  "address": {  "number": 1757,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Video Games", "Base Jumping", "Tennis" ],  "children": [ {  "name": "Madaline Godette",  "age": 10 }, {  "name": "Shasta Godette",  "age": 15 }, {  "name": "Parthenia Godette",  "age": 11 }, {  "name": "Priscila Godette",  "age": 13 } ] }
+{  "cid": 462,  "name": "Margaret Galvis",  "interests": [ "Base Jumping", "Movies", "Movies" ],  "children": [ {  "name": "Isaac Galvis",  "age": 48 }, {  "name": "Mei Galvis" }, {  "name": "Asha Galvis" }, {  "name": "Zachery Galvis" } ] }
+{  "cid": 38,  "name": "Lawanna Abadi",  "age": 35,  "address": {  "number": 6942,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Arthur Abadi",  "age": 10 } ] }
+{  "cid": 362,  "name": "Alta Bantug",  "interests": [ "Computers" ],  "children": [  ] }
+{  "cid": 387,  "name": "Leonard Mabie",  "age": 33,  "address": {  "number": 6703,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Bass", "Running", "Walking" ],  "children": [ {  "name": "Jone Mabie",  "age": 16 }, {  "name": "Claire Mabie" }, {  "name": "Larraine Mabie" }, {  "name": "Corrina Mabie" } ] }
+{  "cid": 56,  "name": "Andria Killelea",  "interests": [ "Cigars", "Skiing" ],  "children": [  ] }
+{  "cid": 369,  "name": "Nickole Dory",  "age": 10,  "address": {  "number": 4761,  "street": "View St.",  "city": "Portland" },  "interests": [ "Walking", "Cooking" ],  "children": [ {  "name": "Annmarie Dory" }, {  "name": "Michele Dory" }, {  "name": "Annamae Dory" }, {  "name": "Flora Dory" } ] }
+{  "cid": 862,  "name": "Constance Bries",  "age": 77,  "address": {  "number": 2585,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Lizzie Bries",  "age": 42 }, {  "name": "Shenika Bries" }, {  "name": "Phillip Bries" } ] }
+{  "cid": 39,  "name": "Brock Froncillo",  "age": 72,  "address": {  "number": 4645,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Base Jumping", "Skiing" ],  "children": [ {  "name": "Cole Froncillo" }, {  "name": "Ivana Froncillo" }, {  "name": "Hugh Froncillo",  "age": 23 } ] }
+{  "cid": 310,  "name": "Lyda Madriz",  "age": 42,  "address": {  "number": 8543,  "street": "Oak St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Databases", "Running", "Cooking" ],  "children": [ {  "name": "Jamila Madriz" }, {  "name": "Micah Madriz" }, {  "name": "Judie Madriz",  "age": 29 }, {  "name": "Joselyn Madriz",  "age": 31 } ] }
+{  "cid": 34,  "name": "Sam Tannahill",  "interests": [ "Books" ],  "children": [  ] }
+{  "cid": 325,  "name": "Ai Tarleton",  "interests": [ "Coffee", "Music" ],  "children": [ {  "name": "Risa Tarleton",  "age": 24 }, {  "name": "Leonila Tarleton" }, {  "name": "Thomasina Tarleton" } ] }
+{  "cid": 392,  "name": "Isiah Nussbaumer",  "interests": [ "Squash" ],  "children": [  ] }
+{  "cid": 447,  "name": "Iris Schoneman",  "age": 34,  "address": {  "number": 7648,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Bass", "Wine", "Puzzles", "Cigars" ],  "children": [ {  "name": "Shemika Schoneman",  "age": 11 }, {  "name": "Maritza Schoneman",  "age": 21 }, {  "name": "Martha Schoneman",  "age": 20 } ] }
+{  "cid": 240,  "name": "Will Marien",  "interests": [ "Basketball", "Music", "Video Games", "Coffee" ],  "children": [ {  "name": "Hue Marien" }, {  "name": "Waltraud Marien" }, {  "name": "Kai Marien",  "age": 15 }, {  "name": "Tracie Marien",  "age": 42 } ] }
+{  "cid": 24,  "name": "Hosea Wilburn",  "interests": [  ],  "children": [  ] }
+{  "cid": 917,  "name": "Jerri Blachowski",  "interests": [ "Skiing" ],  "children": [ {  "name": "Chet Blachowski",  "age": 43 }, {  "name": "Mallory Blachowski" }, {  "name": "Akilah Blachowski" } ] }
+{  "cid": 147,  "name": "Marla Pollan",  "age": 24,  "address": {  "number": 9271,  "street": "Oak St.",  "city": "Portland" },  "interests": [ "Music" ],  "children": [ {  "name": "Song Pollan",  "age": 11 }, {  "name": "Lili Pollan",  "age": 13 }, {  "name": "Shaunte Pollan",  "age": 12 }, {  "name": "Sandie Pollan" } ] }
+{  "cid": 178,  "name": "Athena Kaluna",  "interests": [ "Running", "Computers", "Basketball" ],  "children": [ {  "name": "Rosalba Kaluna",  "age": 48 }, {  "name": "Max Kaluna",  "age": 10 } ] }
+{  "cid": 889,  "name": "Elvis Schoff",  "age": 83,  "address": {  "number": 6724,  "street": "Hill St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Spring Schoff",  "age": 43 }, {  "name": "Davis Schoff",  "age": 55 }, {  "name": "Ryann Schoff",  "age": 58 }, {  "name": "Clarinda Schoff",  "age": 11 } ] }
+{  "cid": 360,  "name": "Billye Grumet",  "age": 82,  "address": {  "number": 7052,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Coffee" ],  "children": [ {  "name": "Linnea Grumet" }, {  "name": "Charline Grumet",  "age": 67 } ] }
+{  "cid": 141,  "name": "Adena Klockars",  "interests": [ "Skiing", "Computers", "Bass", "Cigars" ],  "children": [  ] }
+{  "cid": 364,  "name": "Joni Dazey",  "age": 14,  "address": {  "number": 1237,  "street": "Oak St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Kraig Dazey" } ] }
+{  "cid": 598,  "name": "Venus Peat",  "interests": [ "Coffee", "Walking", "Cigars" ],  "children": [ {  "name": "Antonetta Peat" }, {  "name": "Shane Peat" } ] }
+{  "cid": 376,  "name": "Jeffrey Hegarty",  "interests": [ "Puzzles" ],  "children": [ {  "name": "April Hegarty" }, {  "name": "Wilbur Hegarty" }, {  "name": "Hanh Hegarty" } ] }
+{  "cid": 544,  "name": "Silas Demay",  "age": 69,  "address": {  "number": 447,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Tennis", "Bass" ],  "children": [ {  "name": "Latonya Demay" }, {  "name": "Lissette Demay",  "age": 37 }, {  "name": "Lynell Demay",  "age": 42 }, {  "name": "Mikel Demay",  "age": 17 } ] }
+{  "cid": 538,  "name": "Mack Vollick",  "interests": [ "Base Jumping", "Fishing", "Walking", "Computers" ],  "children": [ {  "name": "Gil Vollick",  "age": 11 }, {  "name": "Marica Vollick" } ] }
+{  "cid": 88,  "name": "Courtney Muckleroy",  "interests": [ "Wine", "Movies", "Skiing" ],  "children": [ {  "name": "Alona Muckleroy",  "age": 30 }, {  "name": "Flora Muckleroy",  "age": 41 }, {  "name": "Angel Muckleroy" }, {  "name": "Daniella Muckleroy" } ] }
+{  "cid": 136,  "name": "Aubrey Kasuboski",  "interests": [ "Cigars" ],  "children": [  ] }
+{  "cid": 91,  "name": "Luna Machen",  "interests": [ "Wine" ],  "children": [ {  "name": "Randal Machen",  "age": 59 }, {  "name": "Emely Machen" } ] }
+{  "cid": 497,  "name": "Chantay Balak",  "interests": [ "Bass", "Fishing" ],  "children": [ {  "name": "John Balak" }, {  "name": "Thu Balak",  "age": 38 } ] }
+{  "cid": 296,  "name": "Doreen Kea",  "age": 89,  "address": {  "number": 7034,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [ "Movies" ],  "children": [ {  "name": "Lyndsay Kea",  "age": 68 }, {  "name": "Trena Kea",  "age": 18 } ] }
+{  "cid": 106,  "name": "Charles Verna",  "interests": [ "Bass", "Books" ],  "children": [ {  "name": "Betsy Verna",  "age": 37 }, {  "name": "Chae Verna",  "age": 35 }, {  "name": "Naoma Verna",  "age": 42 } ] }
+{  "cid": 893,  "name": "Norberto Banchero",  "interests": [  ],  "children": [  ] }
+{  "cid": 953,  "name": "Erasmo Nate",  "interests": [ "Bass", "Cigars", "Books", "Basketball" ],  "children": [ {  "name": "Doloris Nate",  "age": 11 } ] }
+{  "cid": 643,  "name": "Juliet Skreen",  "interests": [ "Walking" ],  "children": [  ] }
+{  "cid": 579,  "name": "Sabra Yuenger",  "age": 45,  "address": {  "number": 2681,  "street": "Cedar St.",  "city": "Sunnyvale" },  "interests": [ "Puzzles" ],  "children": [ {  "name": "Eddie Yuenger" } ] }
+{  "cid": 94,  "name": "Edgardo Dunnegan",  "interests": [  ],  "children": [ {  "name": "Lyndia Dunnegan" } ] }
+{  "cid": 617,  "name": "Jacques Gaskill",  "interests": [ "Cigars", "Coffee", "Computers", "Wine" ],  "children": [ {  "name": "Angelyn Gaskill" }, {  "name": "Jeanett Gaskill",  "age": 40 }, {  "name": "Emelda Gaskill",  "age": 34 } ] }
+{  "cid": 605,  "name": "Sue Henriksen",  "age": 78,  "address": {  "number": 7208,  "street": "Cedar St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Lauretta Henriksen" }, {  "name": "Leigh Henriksen",  "age": 11 } ] }
+{  "cid": 660,  "name": "Israel Aday",  "interests": [ "Wine", "Bass", "Cigars" ],  "children": [ {  "name": "Mi Aday" } ] }
+{  "cid": 71,  "name": "Alva Sieger",  "interests": [ "Movies", "Walking" ],  "children": [ {  "name": "Renetta Sieger" }, {  "name": "Shiloh Sieger",  "age": 57 }, {  "name": "Lavina Sieger" }, {  "name": "Larraine Sieger" } ] }
+{  "cid": 730,  "name": "Marti Vandoren",  "interests": [ "Skiing", "Bass" ],  "children": [ {  "name": "Carroll Vandoren" }, {  "name": "Lorretta Vandoren",  "age": 30 }, {  "name": "Chloe Vandoren",  "age": 42 }, {  "name": "Ilona Vandoren" } ] }
+{  "cid": 388,  "name": "Laree Faist",  "age": 20,  "address": {  "number": 1003,  "street": "Main St.",  "city": "Seattle" },  "interests": [ "Skiing", "Movies", "Video Games", "Cooking" ],  "children": [ {  "name": "Parthenia Faist" }, {  "name": "Maxima Faist" }, {  "name": "Merissa Faist" } ] }
+{  "cid": 958,  "name": "Ricardo Pezzica",  "interests": [  ],  "children": [ {  "name": "Delois Pezzica",  "age": 11 } ] }
+{  "cid": 485,  "name": "Gene Rogoff",  "interests": [ "Fishing" ],  "children": [ {  "name": "Ebonie Rogoff" } ] }
+{  "cid": 470,  "name": "Yesenia Doyon",  "age": 78,  "address": {  "number": 3641,  "street": "7th St.",  "city": "Seattle" },  "interests": [ "Databases", "Puzzles" ],  "children": [ {  "name": "Halley Doyon" }, {  "name": "Teisha Doyon",  "age": 33 }, {  "name": "Warren Doyon" } ] }
+{  "cid": 625,  "name": "Gale Marrazzo",  "age": 25,  "address": {  "number": 2307,  "street": "View St.",  "city": "San Jose" },  "interests": [ "Fishing", "Base Jumping", "Walking", "Cooking" ],  "children": [ {  "name": "Coleman Marrazzo" }, {  "name": "Frances Marrazzo" }, {  "name": "Camellia Marrazzo",  "age": 11 } ] }
+{  "cid": 72,  "name": "Clarissa Geraldes",  "age": 67,  "address": {  "number": 8248,  "street": "Park St.",  "city": "Los Angeles" },  "interests": [ "Cigars", "Walking", "Databases", "Video Games" ],  "children": [ {  "name": "Vina Geraldes",  "age": 51 } ] }
+{  "cid": 188,  "name": "Brynn Bendorf",  "age": 23,  "address": {  "number": 1168,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Skiing" ],  "children": [ {  "name": "Leesa Bendorf",  "age": 11 }, {  "name": "Daine Bendorf" } ] }
+{  "cid": 635,  "name": "Angelena Braegelmann",  "age": 36,  "address": {  "number": 4158,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Wine", "Skiing" ],  "children": [ {  "name": "Daisey Braegelmann",  "age": 18 }, {  "name": "Gaston Braegelmann",  "age": 19 }, {  "name": "Louella Braegelmann" }, {  "name": "Leonie Braegelmann" } ] }
+{  "cid": 595,  "name": "Samuel Brawdy",  "age": 28,  "address": {  "number": 453,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [ "Books", "Basketball" ],  "children": [ {  "name": "Marlen Brawdy",  "age": 14 }, {  "name": "Lorine Brawdy",  "age": 13 }, {  "name": "Brad Brawdy" } ] }
+{  "cid": 916,  "name": "Kris Mcmarlin",  "interests": [ "Movies", "Music", "Puzzles" ],  "children": [  ] }
+{  "cid": 101,  "name": "Meaghan Vandel",  "interests": [ "Music", "Base Jumping", "Books" ],  "children": [ {  "name": "Larissa Vandel" } ] }
+{  "cid": 393,  "name": "Rossana Monton",  "age": 34,  "address": {  "number": 4490,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Skiing", "Base Jumping" ],  "children": [ {  "name": "Glayds Monton" }, {  "name": "Lily Monton" }, {  "name": "Raina Monton" }, {  "name": "Hilma Monton" } ] }
+{  "cid": 857,  "name": "Kasie Fujioka",  "interests": [ "Skiing", "Cigars" ],  "children": [ {  "name": "Leontine Fujioka" }, {  "name": "Nga Fujioka",  "age": 21 }, {  "name": "Nathanael Fujioka",  "age": 27 } ] }
+{  "cid": 57,  "name": "Celestine Mac",  "interests": [ "Wine", "Computers", "Books" ],  "children": [ {  "name": "Kathyrn Mac",  "age": 44 } ] }
+{  "cid": 724,  "name": "Merle Bakula",  "interests": [  ],  "children": [ {  "name": "Margart Bakula",  "age": 49 }, {  "name": "Mathew Bakula",  "age": 36 } ] }
+{  "cid": 407,  "name": "Bebe Cotney",  "interests": [ "Books", "Tennis" ],  "children": [ {  "name": "Daren Cotney" }, {  "name": "Lady Cotney",  "age": 48 } ] }
+{  "cid": 611,  "name": "Evelyne Bassette",  "interests": [ "Coffee" ],  "children": [ {  "name": "Angla Bassette",  "age": 13 } ] }
+{  "cid": 460,  "name": "Jeraldine Choules",  "interests": [ "Fishing" ],  "children": [ {  "name": "Berneice Choules",  "age": 16 }, {  "name": "Jaime Choules",  "age": 21 }, {  "name": "Li Choules",  "age": 20 }, {  "name": "Leah Choules" } ] }
+{  "cid": 120,  "name": "Jan Gianandrea",  "interests": [ "Databases", "Movies", "Cigars" ],  "children": [ {  "name": "Keesha Gianandrea" }, {  "name": "Vashti Gianandrea",  "age": 35 }, {  "name": "Larry Gianandrea",  "age": 29 } ] }
+{  "cid": 81,  "name": "Lavonda Manford",  "age": 87,  "address": {  "number": 2423,  "street": "Main St.",  "city": "San Jose" },  "interests": [  ],  "children": [  ] }
+{  "cid": 421,  "name": "Rubye Dillabough",  "age": 55,  "address": {  "number": 6980,  "street": "View St.",  "city": "Sunnyvale" },  "interests": [ "Squash" ],  "children": [ {  "name": "Hyacinth Dillabough",  "age": 19 }, {  "name": "Arie Dillabough" } ] }
+{  "cid": 875,  "name": "Ramon Crepps",  "interests": [ "Coffee", "Movies", "Skiing" ],  "children": [ {  "name": "Elisha Crepps" } ] }
+{  "cid": 842,  "name": "Omega Vanhoozer",  "age": 67,  "address": {  "number": 7806,  "street": "View St.",  "city": "Portland" },  "interests": [ "Music", "Walking", "Bass", "Wine" ],  "children": [ {  "name": "Lavina Vanhoozer" }, {  "name": "Mike Vanhoozer" } ] }
+{  "cid": 831,  "name": "Raina Rys",  "age": 62,  "address": {  "number": 7048,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [ "Walking" ],  "children": [ {  "name": "Ezra Rys" }, {  "name": "Carl Rys" }, {  "name": "Loraine Rys" } ] }
+{  "cid": 950,  "name": "Young Bayn",  "interests": [  ],  "children": [ {  "name": "Evangeline Bayn",  "age": 38 }, {  "name": "Darcy Bayn",  "age": 45 }, {  "name": "Rosita Bayn" }, {  "name": "Austin Bayn",  "age": 46 } ] }
+{  "cid": 107,  "name": "Abigail Niemiec",  "age": 87,  "address": {  "number": 39,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Tennis", "Databases", "Skiing", "Music" ],  "children": [ {  "name": "Cecil Niemiec",  "age": 66 } ] }
+{  "cid": 702,  "name": "Lane Krog",  "age": 50,  "address": {  "number": 1646,  "street": "Lake St.",  "city": "Mountain View" },  "interests": [ "Running" ],  "children": [ {  "name": "Carri Krog" }, {  "name": "Sage Krog" }, {  "name": "Bronwyn Krog" } ] }
+{  "cid": 764,  "name": "Nakita Sharlow",  "interests": [ "Databases", "Basketball", "Cigars", "Base Jumping" ],  "children": [ {  "name": "Della Sharlow",  "age": 52 }, {  "name": "Horacio Sharlow",  "age": 22 }, {  "name": "Samual Sharlow" } ] }
+{  "cid": 829,  "name": "Donnette Lebel",  "interests": [ "Tennis", "Coffee", "Running", "Fishing" ],  "children": [ {  "name": "Junior Lebel" } ] }
+{  "cid": 370,  "name": "Shonta Furby",  "age": 18,  "address": {  "number": 5792,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [ "Databases" ],  "children": [ {  "name": "Raleigh Furby" }, {  "name": "Britta Furby" }, {  "name": "Gay Furby" }, {  "name": "Elenor Furby" } ] }
+{  "cid": 697,  "name": "Claud Coffel",  "age": 72,  "address": {  "number": 8483,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Katheleen Coffel",  "age": 38 }, {  "name": "Tashina Coffel" } ] }
+{  "cid": 604,  "name": "Clyde Remak",  "interests": [ "Tennis", "Tennis", "Books", "Computers" ],  "children": [ {  "name": "Ward Remak" } ] }
+{  "cid": 342,  "name": "Maxima Cason",  "age": 67,  "address": {  "number": 6644,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Cigars", "Tennis", "Puzzles", "Basketball" ],  "children": [ {  "name": "Alba Cason" } ] }
+{  "cid": 602,  "name": "Clyde Salada",  "age": 59,  "address": {  "number": 8316,  "street": "7th St.",  "city": "Sunnyvale" },  "interests": [ "Movies", "Skiing", "Cooking" ],  "children": [  ] }
+{  "cid": 84,  "name": "Huong Kachel",  "interests": [ "Music", "Tennis", "Base Jumping" ],  "children": [ {  "name": "Katlyn Kachel",  "age": 40 }, {  "name": "Sherman Kachel" }, {  "name": "Susana Kachel",  "age": 32 } ] }
+{  "cid": 784,  "name": "Omar Hasen",  "interests": [ "Movies" ],  "children": [ {  "name": "Hugh Hasen" } ] }
+{  "cid": 451,  "name": "Lelia Sondelski",  "age": 60,  "address": {  "number": 4044,  "street": "Park St.",  "city": "Portland" },  "interests": [ "Books", "Squash", "Walking" ],  "children": [  ] }
+{  "cid": 413,  "name": "Maurice Landrie",  "interests": [ "Computers", "Coffee" ],  "children": [ {  "name": "Gail Landrie",  "age": 37 }, {  "name": "Carylon Landrie" }, {  "name": "Allen Landrie",  "age": 16 }, {  "name": "Andreas Landrie" } ] }
+{  "cid": 156,  "name": "Bobbye Kauppi",  "age": 79,  "address": {  "number": 2051,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Base Jumping", "Cigars", "Movies" ],  "children": [  ] }
+{  "cid": 255,  "name": "Cherri Piegaro",  "age": 64,  "address": {  "number": 3802,  "street": "Oak St.",  "city": "Sunnyvale" },  "interests": [  ],  "children": [ {  "name": "Elwood Piegaro" } ] }
+{  "cid": 343,  "name": "Kaylee Ozaine",  "age": 78,  "address": {  "number": 3367,  "street": "Washington St.",  "city": "Seattle" },  "interests": [  ],  "children": [ {  "name": "Darwin Ozaine",  "age": 35 }, {  "name": "Anne Ozaine",  "age": 13 }, {  "name": "Kenneth Ozaine" }, {  "name": "Pat Ozaine",  "age": 53 } ] }
+{  "cid": 898,  "name": "Thao Seufert",  "age": 78,  "address": {  "number": 3529,  "street": "Hill St.",  "city": "Seattle" },  "interests": [ "Bass", "Squash", "Coffee" ],  "children": [ {  "name": "Classie Seufert" } ] }
+{  "cid": 301,  "name": "Cherry Steenwyk",  "age": 88,  "address": {  "number": 4138,  "street": "Lake St.",  "city": "San Jose" },  "interests": [ "Movies" ],  "children": [ {  "name": "Toccara Steenwyk",  "age": 66 }, {  "name": "Tari Steenwyk" }, {  "name": "Lawanna Steenwyk" }, {  "name": "Ossie Steenwyk",  "age": 26 } ] }
+{  "cid": 465,  "name": "Rey Arango",  "age": 68,  "address": {  "number": 1788,  "street": "View St.",  "city": "Los Angeles" },  "interests": [ "Tennis" ],  "children": [  ] }
+{  "cid": 775,  "name": "Jerry Lowing",  "age": 62,  "address": {  "number": 1055,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Puzzles", "Books", "Running", "Bass" ],  "children": [ {  "name": "Emmitt Lowing" }, {  "name": "Kimberly Lowing" } ] }
+{  "cid": 340,  "name": "Erick Faiola",  "interests": [ "Coffee" ],  "children": [ {  "name": "Marquita Faiola" }, {  "name": "Tasia Faiola" }, {  "name": "Micheal Faiola",  "age": 24 }, {  "name": "Salvatore Faiola" } ] }
+{  "cid": 577,  "name": "Alejandro Oblinger",  "interests": [ "Movies", "Movies" ],  "children": [ {  "name": "Tenesha Oblinger",  "age": 56 }, {  "name": "Loni Oblinger",  "age": 12 }, {  "name": "Sherryl Oblinger" } ] }
+{  "cid": 838,  "name": "Karan Aharon",  "age": 88,  "address": {  "number": 8033,  "street": "Washington St.",  "city": "Portland" },  "interests": [ "Computers", "Movies", "Walking" ],  "children": [ {  "name": "Matha Aharon",  "age": 16 } ] }
+{  "cid": 235,  "name": "Orpha Craycraft",  "interests": [ "Skiing", "Squash" ],  "children": [  ] }
+{  "cid": 868,  "name": "Berry Steward",  "age": 12,  "address": {  "number": 8594,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Fishing", "Tennis", "Movies", "Video Games" ],  "children": [ {  "name": "Mason Steward" }, {  "name": "Yoshiko Steward" }, {  "name": "Toni Steward" } ] }
+{  "cid": 153,  "name": "Randy Hueso",  "age": 11,  "address": {  "number": 1957,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Computers", "Wine", "Databases", "Walking" ],  "children": [  ] }
+{  "cid": 82,  "name": "Gloria Junkins",  "interests": [ "Basketball" ],  "children": [  ] }
+{  "cid": 990,  "name": "Javier Searer",  "age": 38,  "address": {  "number": 3817,  "street": "Park St.",  "city": "Sunnyvale" },  "interests": [ "Databases", "Cigars", "Fishing", "Basketball" ],  "children": [ {  "name": "Griselda Searer",  "age": 13 }, {  "name": "Josephina Searer",  "age": 27 }, {  "name": "Brice Searer",  "age": 22 }, {  "name": "Kelly Searer" } ] }
+{  "cid": 89,  "name": "Calandra Hedden",  "age": 33,  "address": {  "number": 1231,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Wine" ],  "children": [ {  "name": "Damien Hedden",  "age": 19 } ] }
+{  "cid": 171,  "name": "Eddie Shebchuk",  "age": 86,  "address": {  "number": 3304,  "street": "Lake St.",  "city": "Portland" },  "interests": [ "Books" ],  "children": [ {  "name": "Harmony Shebchuk" } ] }
+{  "cid": 513,  "name": "Marianna Gortman",  "age": 49,  "address": {  "number": 927,  "street": "Cedar St.",  "city": "San Jose" },  "interests": [ "Databases", "Databases" ],  "children": [  ] }
+{  "cid": 907,  "name": "Princess Sudol",  "age": 73,  "address": {  "number": 9770,  "street": "Oak St.",  "city": "San Jose" },  "interests": [ "Computers", "Base Jumping" ],  "children": [ {  "name": "Bronwyn Sudol",  "age": 22 }, {  "name": "Judith Sudol" } ] }
+{  "cid": 627,  "name": "Fernande Ede",  "age": 75,  "address": {  "number": 9316,  "street": "Cedar St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Rebeca Ede" }, {  "name": "Raymond Ede",  "age": 57 } ] }
+{  "cid": 647,  "name": "Jodi Dearson",  "interests": [ "Fishing", "Movies" ],  "children": [  ] }
+{  "cid": 111,  "name": "Eddy Ortea",  "age": 16,  "address": {  "number": 6874,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Shera Ortea" } ] }
+{  "cid": 830,  "name": "Laurice Halik",  "interests": [ "Puzzles", "Tennis", "Tennis", "Books" ],  "children": [ {  "name": "Bobby Halik" }, {  "name": "Stormy Halik" } ] }
+{  "cid": 331,  "name": "Willena Provenza",  "age": 43,  "address": {  "number": 6742,  "street": "Main St.",  "city": "Portland" },  "interests": [ "Basketball" ],  "children": [ {  "name": "Alesha Provenza",  "age": 32 }, {  "name": "Marty Provenza" }, {  "name": "Lindy Provenza",  "age": 21 }, {  "name": "Junita Provenza" } ] }
+{  "cid": 849,  "name": "Kristen Zapalac",  "age": 14,  "address": {  "number": 4087,  "street": "Lake St.",  "city": "Sunnyvale" },  "interests": [ "Wine", "Cooking", "Running", "Computers" ],  "children": [  ] }
+{  "cid": 700,  "name": "Suk Blondin",  "interests": [ "Wine" ],  "children": [ {  "name": "Brenton Blondin" }, {  "name": "Charlotte Blondin" }, {  "name": "Eldon Blondin",  "age": 10 }, {  "name": "Leanne Blondin" } ] }
+{  "cid": 481,  "name": "Leana Revera",  "interests": [ "Running", "Skiing" ],  "children": [ {  "name": "Marquita Revera" } ] }
+{  "cid": 306,  "name": "Laurie Tuff",  "interests": [ "Computers", "Base Jumping", "Bass", "Basketball" ],  "children": [ {  "name": "Sharie Tuff" }, {  "name": "Ollie Tuff",  "age": 53 }, {  "name": "Gonzalo Tuff" }, {  "name": "Thomas Tuff" } ] }
+{  "cid": 533,  "name": "Trinity Urquidez",  "interests": [  ],  "children": [ {  "name": "Corrine Urquidez",  "age": 29 }, {  "name": "Markita Urquidez",  "age": 19 }, {  "name": "Danette Urquidez" } ] }
+{  "cid": 968,  "name": "Alix Levier",  "age": 44,  "address": {  "number": 7241,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [ "Databases", "Fishing", "Wine" ],  "children": [ {  "name": "Florentina Levier" }, {  "name": "Hyon Levier" }, {  "name": "Dannielle Levier" } ] }
+{  "cid": 834,  "name": "Luvenia Grandstaff",  "interests": [ "Squash" ],  "children": [ {  "name": "Joleen Grandstaff",  "age": 28 }, {  "name": "Elvera Grandstaff" }, {  "name": "Leonia Grandstaff",  "age": 35 }, {  "name": "Jaclyn Grandstaff",  "age": 28 } ] }
+{  "cid": 129,  "name": "Marisha Canzoneri",  "age": 84,  "address": {  "number": 5507,  "street": "View St.",  "city": "Mountain View" },  "interests": [ "Music", "Databases", "Walking", "Walking" ],  "children": [  ] }
+{  "cid": 735,  "name": "Lonnie Bechel",  "age": 36,  "address": {  "number": 592,  "street": "Main St.",  "city": "Sunnyvale" },  "interests": [ "Walking", "Cigars", "Squash", "Wine" ],  "children": [  ] }
+{  "cid": 358,  "name": "Fredricka Krum",  "interests": [  ],  "children": [ {  "name": "Darrick Krum" }, {  "name": "Julieann Krum" }, {  "name": "Sun Krum" }, {  "name": "Rosamaria Krum",  "age": 16 } ] }
+{  "cid": 937,  "name": "Annika Pauline",  "age": 78,  "address": {  "number": 8563,  "street": "Hill St.",  "city": "Los Angeles" },  "interests": [  ],  "children": [ {  "name": "Mikki Pauline",  "age": 34 } ] }
+{  "cid": 183,  "name": "Ladawn Vyas",  "age": 64,  "address": {  "number": 2663,  "street": "View St.",  "city": "Portland" },  "interests": [  ],  "children": [  ] }
+{  "cid": 738,  "name": "Josphine Rohrer",  "age": 75,  "address": {  "number": 862,  "street": "Main St.",  "city": "Los Angeles" },  "interests": [ "Databases" ],  "children": [ {  "name": "Marvin Rohrer",  "age": 22 }, {  "name": "Wyatt Rohrer" }, {  "name": "Deloras Rohrer" } ] }
+{  "cid": 599,  "name": "Alva Molaison",  "age": 87,  "address": {  "number": 5974,  "street": "Washington St.",  "city": "Seattle" },  "interests": [ "Wine", "Squash" ],  "children": [ {  "name": "Milo Molaison",  "age": 39 } ] }
+{  "cid": 167,  "name": "Philomena Alsop",  "age": 45,  "address": {  "number": 9468,  "street": "7th St.",  "city": "Mountain View" },  "interests": [ "Cigars", "Walking", "Tennis", "Base Jumping" ],  "children": [ {  "name": "Antoinette Alsop",  "age": 13 }, {  "name": "Emile Alsop" } ] }
+{  "cid": 986,  "name": "Tennille Wikle",  "age": 78,  "address": {  "number": 3428,  "street": "View St.",  "city": "Portland" },  "interests": [ "Movies", "Databases", "Wine" ],  "children": [ {  "name": "Lourie Wikle" }, {  "name": "Laure Wikle" } ] }
+{  "cid": 96,  "name": "Mara Aument",  "age": 72,  "address": {  "number": 7709,  "street": "Hill St.",  "city": "Sunnyvale" },  "interests": [ "Cigars", "Cooking", "Movies" ],  "children": [ {  "name": "Leonardo Aument",  "age": 22 } ] }
+{  "cid": 130,  "name": "Kandis Hissem",  "interests": [ "Tennis" ],  "children": [ {  "name": "Arianna Hissem" }, {  "name": "Necole Hissem",  "age": 53 }, {  "name": "Manie Hissem" }, {  "name": "Deshawn Hissem",  "age": 27 } ] }
+{  "cid": 259,  "name": "Aurelio Darrigo",  "age": 45,  "address": {  "number": 1114,  "street": "Park St.",  "city": "San Jose" },  "interests": [ "Cooking", "Running" ],  "children": [ {  "name": "Leonard Darrigo",  "age": 22 }, {  "name": "Aron Darrigo" }, {  "name": "Pamelia Darrigo",  "age": 14 } ] }
+{  "cid": 221,  "name": "Delois Fiqueroa",  "interests": [  ],  "children": [ {  "name": "Cherri Fiqueroa" } ] }
+{  "cid": 207,  "name": "Phyliss Honda",  "age": 22,  "address": {  "number": 8387,  "street": "Lake St.",  "city": "Seattle" },  "interests": [ "Cooking", "Music", "Books" ],  "children": [ {  "name": "Bee Honda" }, {  "name": "Cyril Honda" }, {  "name": "Vertie Honda" } ] }
+{  "cid": 947,  "name": "Fernande Shogren",  "age": 10,  "address": {  "number": 3449,  "street": "Lake St.",  "city": "Los Angeles" },  "interests": [ "Cooking", "Puzzles", "Music", "Squash" ],  "children": [ {  "name": "Buford Shogren" }, {  "name": "Verla Shogren" }, {  "name": "Stefania Shogren" }, {  "name": "Annika Shogren" } ] }
+{  "cid": 368,  "name": "Tequila Scandalios",  "interests": [  ],  "children": [ {  "name": "Nilsa Scandalios" }, {  "name": "Kaye Scandalios",  "age": 23 }, {  "name": "Angelo Scandalios",  "age": 24 } ] }
+{  "cid": 930,  "name": "Kathie Gier",  "age": 37,  "address": {  "number": 5075,  "street": "Main St.",  "city": "Portland" },  "interests": [  ],  "children": [ {  "name": "Onie Gier",  "age": 16 } ] }
+{  "cid": 867,  "name": "Denise Dipiero",  "interests": [ "Basketball", "Cigars", "Cooking", "Running" ],  "children": [ {  "name": "Santa Dipiero" } ] }
+{  "cid": 552,  "name": "Marlena Humann",  "interests": [  ],  "children": [  ] }
+{  "cid": 126,  "name": "Grayce Keir",  "interests": [ "Wine" ],  "children": [ {  "name": "Antonia Keir",  "age": 25 } ] }
+{  "cid": 789,  "name": "Carli Notto",  "interests": [ "Cigars" ],  "children": [  ] }
+{  "cid": 967,  "name": "Melida Laliotis",  "interests": [ "Music", "Base Jumping", "Coffee", "Books" ],  "children": [ {  "name": "Lai Laliotis",  "age": 52 }, {  "name": "Jillian Laliotis",  "age": 11 } ] }
+{  "cid": 203,  "name": "Elke Mazurowski",  "age": 52,  "address": {  "number": 9276,  "street": "View St.",  "city": "Mountain View" },  "interests": [  ],  "children": [ {  "name": "Esta Mazurowski" }, {  "name": "Clarence Mazurowski",  "age": 14 } ] }
+{  "cid": 654,  "name": "Louis Laubersheimer",  "age": 76,  "address": {  "number": 8010,  "street": "7th St.",  "city": "San Jose" },  "interests": [ "Base Jumping", "Bass", "Cooking" ],  "children": [ {  "name": "Jewel Laubersheimer",  "age": 22 }, {  "name": "Toccara Laubersheimer",  "age": 45 }, {  "name": "Eve Laubersheimer" } ] }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index 25aea8f..eae4c46 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -509,8 +509,8 @@
         }
 
         AlgebricksPartitionConstraint clusterLocs = planAndMetadata.getClusterLocations();
-        builder.setBinaryBooleanInspector(format.getBinaryBooleanInspector());
-        builder.setBinaryIntegerInspector(format.getBinaryIntegerInspector());
+        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
+        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
         builder.setClusterLocations(clusterLocs);
         builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
         builder.setExpressionRuntimeProvider(new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(
@@ -555,9 +555,8 @@
             DisplayFormat pdf) throws Exception {
         for (int i = 0; i < specs.length; i++) {
             specs[i].setMaxReattempts(0);
-            JobId jobId = hcc.createJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
+            JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
             long startTime = System.currentTimeMillis();
-            hcc.start(jobId);
             hcc.waitForCompletion(jobId);
             long endTime = System.currentTimeMillis();
             double duration = (endTime - startTime) / 1000.00;
@@ -570,10 +569,9 @@
             throws Exception {
         for (int i = 0; i < jobs.length; i++) {
             jobs[i].getJobSpec().setMaxReattempts(0);
-            JobId jobId = hcc.createJob(GlobalConfig.HYRACKS_APP_NAME, jobs[i].getJobSpec());
             long startTime = System.currentTimeMillis();
             try {
-                hcc.start(jobId);
+                JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, jobs[i].getJobSpec());
                 if (jobs[i].getSubmissionMode() == SubmissionMode.ASYNCHRONOUS) {
                     continue;
                 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 5f4c189..541edd0 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -22,7 +22,6 @@
 
     public static final int DEFAULT_HYRACKS_CC_CLUSTER_PORT = 1099;
 
-    public static final int FRAME_SIZE = 32768;
 
     private static ClusterControllerService cc;
     private static NodeControllerService nc1;
@@ -46,7 +45,6 @@
         ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
-        ncConfig1.frameSize = FRAME_SIZE;
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
 
@@ -56,7 +54,6 @@
         ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
-        ncConfig2.frameSize = FRAME_SIZE;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
@@ -84,9 +81,8 @@
     }
 
     public static void runJob(JobSpecification spec) throws Exception {
-        JobId jobId = hcc.createJob(GlobalConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         GlobalConfig.ASTERIX_LOGGER.info(spec.toJSON().toString());
-        hcc.start(jobId);
+        JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         GlobalConfig.ASTERIX_LOGGER.info(jobId.toString());
         hcc.waitForCompletion(jobId);
     }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
index 5e891c8..c68817c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
@@ -93,925 +93,1090 @@
 
 public class DdlTranslator extends AbstractAqlTranslator {
 
-    private final MetadataTransactionContext mdTxnCtx;
-    private final List<Statement> aqlStatements;
-    private final PrintWriter out;
-    private final SessionConfig pc;
-    private final DisplayFormat pdf;
-    private AqlCompiledMetadataDeclarations compiledDeclarations;
+	private final MetadataTransactionContext mdTxnCtx;
+	private final List<Statement> aqlStatements;
+	private final PrintWriter out;
+	private final SessionConfig pc;
+	private final DisplayFormat pdf;
+	private AqlCompiledMetadataDeclarations compiledDeclarations;
 
-    private static Map<String, BuiltinType> builtinTypeMap;
+	private static Map<String, BuiltinType> builtinTypeMap;
 
-    public DdlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements, PrintWriter out,
-            SessionConfig pc, DisplayFormat pdf) {
-        this.mdTxnCtx = mdTxnCtx;
-        this.aqlStatements = aqlStatements;
-        this.out = out;
-        this.pc = pc;
-        this.pdf = pdf;
-        builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
-    }
+	public DdlTranslator(MetadataTransactionContext mdTxnCtx,
+			List<Statement> aqlStatements, PrintWriter out, SessionConfig pc,
+			DisplayFormat pdf) {
+		this.mdTxnCtx = mdTxnCtx;
+		this.aqlStatements = aqlStatements;
+		this.out = out;
+		this.pc = pc;
+		this.pdf = pdf;
+		builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
+	}
 
-    public void translate(IHyracksClientConnection hcc, boolean disconnectFromDataverse) throws AlgebricksException {
-        try {
-            compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements, true);
-            compileAndExecuteDDLstatements(hcc, mdTxnCtx, disconnectFromDataverse);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-    }
+	public void translate(IHyracksClientConnection hcc,
+			boolean disconnectFromDataverse) throws AlgebricksException {
+		try {
+			compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements,
+					true);
+			compileAndExecuteDDLstatements(hcc, mdTxnCtx,
+					disconnectFromDataverse);
+		} catch (Exception e) {
+			throw new AlgebricksException(e);
+		}
+	}
 
-    private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
-            boolean disconnectFromDataverse) throws Exception {
-        for (Statement stmt : aqlStatements) {
-            validateOperation(compiledDeclarations, stmt);
-            switch (stmt.getKind()) {
-                case DATAVERSE_DECL: {
-                    checkForDataverseConnection(false);
-                    DataverseDecl dvd = (DataverseDecl) stmt;
-                    String dataverseName = dvd.getDataverseName().getValue();
-                    compiledDeclarations.connectToDataverse(dataverseName);
-                    break;
-                }
-                
-                case CREATE_DATAVERSE: {
-                    checkForDataverseConnection(false);
-                    CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
-                    String dvName = stmtCreateDataverse.getDataverseName().getValue();
-                    Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
-                    if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
-                            throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
-                    }
-                    MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
-                    		new Dataverse(dvName, stmtCreateDataverse.getFormat()));
-                    break;
-                }
-                
-                case DATASET_DECL: {
-                    checkForDataverseConnection(true);
-                    DatasetDecl dd = (DatasetDecl) stmt;
-                    String datasetName = dd.getName().getValue();
-                    DatasetType dsType = dd.getDatasetType();
-                    String itemTypeName = null;
-                    IDatasetDetails datasetDetails = null;
-                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                            datasetName);
-                    if (ds != null) {
-                        if (dd.getIfNotExists()) {
-                            continue;                            
-                        } else {
-                            throw new AlgebricksException("A dataset with this name " + datasetName
-                                    + " already exists.");
-                        }
-                    }
-                    itemTypeName = dd.getItemTypeName().getValue();
-                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-                            compiledDeclarations.getDataverseName(), itemTypeName);
-                    if (dt == null) {
-                        throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
-                    }
-                    switch (dd.getDatasetType()) {
-                        case INTERNAL: {
-                            IAType itemType = dt.getDatatype();
-                            if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                                throw new AlgebricksException("Can only partition ARecord's.");
-                            }
-                            List<String> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
-                                    .getPartitioningExprs();
-                            String ngName = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
-                                    .getValue();
-                            datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
-                                    InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
-                                    partitioningExprs, ngName);
-                            break;
-                        }
-                        case EXTERNAL: {
-                            String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
-                            Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl())
-                                    .getProperties();
-                            datasetDetails = new ExternalDatasetDetails(adapter, properties);
-                            break;
-                        }
-                        case FEED: {
-                            IAType itemType = dt.getDatatype();
-                            if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                                throw new AlgebricksException("Can only partition ARecord's.");
-                            }
-                            List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                                    .getPartitioningExprs();
-                            String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
-                                    .getValue();
-                            String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterClassname();
-                            Map<String, String> properties = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                                    .getProperties();
-                            String functionIdentifier = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
-                                    .getFunctionIdentifier();
-                            datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
-                                    InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
-                                    partitioningExprs, ngName, adapter, properties, functionIdentifier,
-                                    FeedDatasetDetails.FeedState.INACTIVE.toString());
-                            break;
-                        }
-                    }
-                    MetadataManager.INSTANCE.addDataset(mdTxnCtx,
-                            new Dataset(compiledDeclarations.getDataverseName(), datasetName, itemTypeName,
-                                    datasetDetails, dsType));
-                    if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
-                        runCreateDatasetJob(hcc, datasetName);
-                    }
-                    break;
-                }
+	private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc,
+			MetadataTransactionContext mdTxnCtx, boolean disconnectFromDataverse)
+			throws Exception {
+		for (Statement stmt : aqlStatements) {
+			validateOperation(compiledDeclarations, stmt);
+			switch (stmt.getKind()) {
+			case DATAVERSE_DECL: {
+				checkForDataverseConnection(false);
+				DataverseDecl dvd = (DataverseDecl) stmt;
+				String dataverseName = dvd.getDataverseName().getValue();
+				compiledDeclarations.connectToDataverse(dataverseName);
+				break;
+			}
 
-                case CREATE_INDEX: {
-                    checkForDataverseConnection(true);
-                    CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
-                    String datasetName = stmtCreateIndex.getDatasetName().getValue();
-                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                            datasetName);
-                    if (ds == null) {
-                        throw new AlgebricksException("There is no dataset with this name " + datasetName);
-                    }
-                    String indexName = stmtCreateIndex.getIndexName().getValue();
-                    Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                            datasetName, indexName);
-                    if (idx != null) {
-                        if (!stmtCreateIndex.getIfNotExists()) {
-                            throw new AlgebricksException("An index with this name " + indexName + " already exists.");
-                        } else {
-                            stmtCreateIndex.setNeedToCreate(false);
-                        }
-                    } else {
-                        MetadataManager.INSTANCE.addIndex(mdTxnCtx, new Index(compiledDeclarations.getDataverseName(),
-                                datasetName, indexName, stmtCreateIndex.getIndexType(),
-                                stmtCreateIndex.getFieldExprs(), false));
-                        runCreateIndexJob(hcc, stmtCreateIndex);
-                    }                            
-                    break;
-                }
-                case TYPE_DECL: {
-                    checkForDataverseConnection(true);
-                    TypeDecl stmtCreateType = (TypeDecl) stmt;
-                    String typeName = stmtCreateType.getIdent().getValue();
-                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-                            compiledDeclarations.getDataverseName(), typeName);
-                    if (dt != null) {
-                        if (!stmtCreateType.getIfNotExists())
-                            throw new AlgebricksException("A datatype with this name " + typeName
-                                    + " already exists.");
-                    } else {
-                        if (builtinTypeMap.get(typeName) != null) {
-                            throw new AlgebricksException("Cannot redefine builtin type " + typeName + ".");
-                        } else {
-                            Map<String, IAType> typeMap = computeTypes(mdTxnCtx, (TypeDecl) stmt);
-                            IAType type = typeMap.get(typeName);
-                            MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
-                                    new Datatype(compiledDeclarations.getDataverseName(), typeName, type, false));
-                        }
-                    }
-                    break;
-                }
-                case NODEGROUP_DECL: {
-                    NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
-                    String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
-                    NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
-                    if (ng != null) {
-                        if (!stmtCreateNodegroup.getIfNotExists())
-                            throw new AlgebricksException("A nodegroup with this name " + ngName + " already exists.");
-                    } else {
-                        List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
-                        List<String> ncNames = new ArrayList<String>(ncIdentifiers.size());
-                        for (Identifier id : ncIdentifiers) {
-                            ncNames.add(id.getValue());
-                        }
-                        MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(ngName, ncNames));
-                    }
-                    break;
-                }
-                // drop statements
-                case DATAVERSE_DROP: {
-                    DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
-                    String dvName = stmtDelete.getDataverseName().getValue();
-                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATAVERSE, dvName)) {
-                        throw new AsterixException("Invalid Operation cannot drop dataverse " + dvName
-                                + " (protected by system)");
-                    }
+			case CREATE_DATAVERSE: {
+				checkForDataverseConnection(false);
+				CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+				String dvName = stmtCreateDataverse.getDataverseName()
+						.getValue();
+				Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
+						dvName);
+				if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
+					throw new AlgebricksException("A dataverse with this name "
+							+ dvName + " already exists.");
+				}
+				MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(
+						dvName, stmtCreateDataverse.getFormat()));
+				break;
+			}
 
-                    if (compiledDeclarations.isConnectedToDataverse())
-                        compiledDeclarations.disconnectFromDataverse();
-                    checkForDataverseConnection(false);
+			case DATASET_DECL: {
+				checkForDataverseConnection(true);
+				DatasetDecl dd = (DatasetDecl) stmt;
+				String datasetName = dd.getName().getValue();
+				DatasetType dsType = dd.getDatasetType();
+				String itemTypeName = null;
+				IDatasetDetails datasetDetails = null;
+				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), datasetName);
+				if (ds != null) {
+					if (dd.getIfNotExists()) {
+						continue;
+					} else {
+						throw new AlgebricksException(
+								"A dataset with this name " + datasetName
+										+ " already exists.");
+					}
+				}
+				itemTypeName = dd.getItemTypeName().getValue();
+				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), itemTypeName);
+				if (dt == null) {
+					throw new AlgebricksException(": type " + itemTypeName
+							+ " could not be found.");
+				}
+				switch (dd.getDatasetType()) {
+				case INTERNAL: {
+					IAType itemType = dt.getDatatype();
+					if (itemType.getTypeTag() != ATypeTag.RECORD) {
+						throw new AlgebricksException(
+								"Can only partition ARecord's.");
+					}
+					List<String> partitioningExprs = ((InternalDetailsDecl) dd
+							.getDatasetDetailsDecl()).getPartitioningExprs();
+					String ngName = ((InternalDetailsDecl) dd
+							.getDatasetDetailsDecl()).getNodegroupName()
+							.getValue();
+					datasetDetails = new InternalDatasetDetails(
+							InternalDatasetDetails.FileStructure.BTREE,
+							InternalDatasetDetails.PartitioningStrategy.HASH,
+							partitioningExprs, partitioningExprs, ngName);
+					break;
+				}
+				case EXTERNAL: {
+					String adapter = ((ExternalDetailsDecl) dd
+							.getDatasetDetailsDecl()).getAdapter();
+					Map<String, String> properties = ((ExternalDetailsDecl) dd
+							.getDatasetDetailsDecl()).getProperties();
+					datasetDetails = new ExternalDatasetDetails(adapter,
+							properties);
+					break;
+				}
+				case FEED: {
+					IAType itemType = dt.getDatatype();
+					if (itemType.getTypeTag() != ATypeTag.RECORD) {
+						throw new AlgebricksException(
+								"Can only partition ARecord's.");
+					}
+					List<String> partitioningExprs = ((FeedDetailsDecl) dd
+							.getDatasetDetailsDecl()).getPartitioningExprs();
+					String ngName = ((FeedDetailsDecl) dd
+							.getDatasetDetailsDecl()).getNodegroupName()
+							.getValue();
+					String adapter = ((FeedDetailsDecl) dd
+							.getDatasetDetailsDecl()).getAdapterClassname();
+					Map<String, String> properties = ((FeedDetailsDecl) dd
+							.getDatasetDetailsDecl()).getProperties();
+					String functionIdentifier = ((FeedDetailsDecl) dd
+							.getDatasetDetailsDecl()).getFunctionIdentifier();
+					datasetDetails = new FeedDatasetDetails(
+							InternalDatasetDetails.FileStructure.BTREE,
+							InternalDatasetDetails.PartitioningStrategy.HASH,
+							partitioningExprs, partitioningExprs, ngName,
+							adapter, properties, functionIdentifier,
+							FeedDatasetDetails.FeedState.INACTIVE.toString());
+					break;
+				}
+				}
+				MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(
+						compiledDeclarations.getDataverseName(), datasetName,
+						itemTypeName, datasetDetails, dsType));
+				if (dd.getDatasetType() == DatasetType.INTERNAL
+						|| dd.getDatasetType() == DatasetType.FEED) {
+					runCreateDatasetJob(hcc, datasetName);
+				}
+				break;
+			}
 
-                    Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
-                    if (dv == null) {
-                        if (!stmtDelete.getIfExists()) {
-                            throw new AlgebricksException("There is no dataverse with this name " + dvName + ".");
-                        }
-                    } else {
-                        compiledDeclarations.connectToDataverse(dvName);
-                        List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
-                        for (int j = 0; j < datasets.size(); j++) {
-                            String datasetName = datasets.get(j).getDatasetName();
-                            DatasetType dsType = datasets.get(j).getType();
-                            if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
-                                List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dvName,
-                                        datasetName);
-                                for (int k = 0; k < indexes.size(); k++) {
-                                    if (indexes.get(k).isSecondaryIndex()) {
-                                        compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(k)
-                                                .getIndexName());
-                                    }
-                                }
-                            }
-                            compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
-                        }
-                        MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
-                        if (compiledDeclarations.isConnectedToDataverse())
-                            compiledDeclarations.disconnectFromDataverse();
-                    }
-                    break;
-                }
-                case DATASET_DROP: {
-                    checkForDataverseConnection(true);
-                    DropStatement stmtDelete = (DropStatement) stmt;
-                    String datasetName = stmtDelete.getDatasetName().getValue();
-                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATASET, datasetName)) {
-                        throw new AsterixException("Invalid Operation cannot drop dataset " + datasetName
-                                + " (protected by system)");
-                    }
-                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                            datasetName);
-                    if (ds == null) {
-                        if (!stmtDelete.getIfExists())
-                            throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
-                    } else {
-                        if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
-                            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
-                                    compiledDeclarations.getDataverseName(), datasetName);
-                            for (int j = 0; j < indexes.size(); j++) {
-                                if (indexes.get(j).isPrimaryIndex()) {
-                                    compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(j).getIndexName());
-                                }
-                            }
-                        }
-                        compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
-                    }
-                    break;
-                }
-                case INDEX_DROP: {
-                    checkForDataverseConnection(true);
-                    IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
-                    String datasetName = stmtDelete.getDatasetName().getValue();
-                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                            datasetName);
-                    if (ds == null)
-                        throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
-                    if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
-                        String indexName = stmtDelete.getIndexName().getValue();
-                        Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
-                                compiledDeclarations.getDataverseName(), datasetName, indexName);
-                        if (idx == null) {
-                            if (!stmtDelete.getIfExists())
-                                throw new AlgebricksException("There is no index with this name " + indexName + ".");
-                        } else
-                            compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexName);
-                    } else {
-                        throw new AlgebricksException(datasetName
-                                + " is an external dataset. Indexes are not maintained for external datasets.");
-                    }
-                    break;
-                }
-                case TYPE_DROP: {
-                    checkForDataverseConnection(true);
-                    TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
-                    String typeName = stmtDelete.getTypeName().getValue();
-                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-                            compiledDeclarations.getDataverseName(), typeName);
-                    if (dt == null) {
-                        if (!stmtDelete.getIfExists())
-                            throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
-                    } else
-                        MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                                typeName);
-                    break;
-                }
-                case NODEGROUP_DROP: {
-                    NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
-                    String nodegroupName = stmtDelete.getNodeGroupName().getValue();
-                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
-                        throw new AsterixException("Invalid Operation cannot drop nodegroup " + nodegroupName
-                                + " (protected by system)");
-                    }
-                    NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
-                    if (ng == null) {
-                        if (!stmtDelete.getIfExists())
-                            throw new AlgebricksException("There is no nodegroup with this name " + nodegroupName
-                                    + ".");
-                    } else
-                        MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodegroupName);
-                    break;
-                }
+			case CREATE_INDEX: {
+				checkForDataverseConnection(true);
+				CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+				String datasetName = stmtCreateIndex.getDatasetName()
+						.getValue();
+				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), datasetName);
+				if (ds == null) {
+					throw new AlgebricksException(
+							"There is no dataset with this name " + datasetName);
+				}
+				String indexName = stmtCreateIndex.getIndexName().getValue();
+				Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), datasetName,
+						indexName);
+				if (idx != null) {
+					if (!stmtCreateIndex.getIfNotExists()) {
+						throw new AlgebricksException(
+								"An index with this name " + indexName
+										+ " already exists.");
+					} else {
+						stmtCreateIndex.setNeedToCreate(false);
+					}
+				} else {
+					MetadataManager.INSTANCE.addIndex(
+							mdTxnCtx,
+							new Index(compiledDeclarations.getDataverseName(),
+									datasetName, indexName, stmtCreateIndex
+											.getIndexType(), stmtCreateIndex
+											.getFieldExprs(), stmtCreateIndex
+											.getGramLength(), false));
+					runCreateIndexJob(hcc, stmtCreateIndex);
+				}
+				break;
+			}
+			case TYPE_DECL: {
+				checkForDataverseConnection(true);
+				TypeDecl stmtCreateType = (TypeDecl) stmt;
+				String typeName = stmtCreateType.getIdent().getValue();
+				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), typeName);
+				if (dt != null) {
+					if (!stmtCreateType.getIfNotExists())
+						throw new AlgebricksException(
+								"A datatype with this name " + typeName
+										+ " already exists.");
+				} else {
+					if (builtinTypeMap.get(typeName) != null) {
+						throw new AlgebricksException(
+								"Cannot redefine builtin type " + typeName
+										+ ".");
+					} else {
+						Map<String, IAType> typeMap = computeTypes(mdTxnCtx,
+								(TypeDecl) stmt);
+						IAType type = typeMap.get(typeName);
+						MetadataManager.INSTANCE.addDatatype(
+								mdTxnCtx,
+								new Datatype(compiledDeclarations
+										.getDataverseName(), typeName, type,
+										false));
+					}
+				}
+				break;
+			}
+			case NODEGROUP_DECL: {
+				NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
+				String ngName = stmtCreateNodegroup.getNodegroupName()
+						.getValue();
+				NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
+						ngName);
+				if (ng != null) {
+					if (!stmtCreateNodegroup.getIfNotExists())
+						throw new AlgebricksException(
+								"A nodegroup with this name " + ngName
+										+ " already exists.");
+				} else {
+					List<Identifier> ncIdentifiers = stmtCreateNodegroup
+							.getNodeControllerNames();
+					List<String> ncNames = new ArrayList<String>(
+							ncIdentifiers.size());
+					for (Identifier id : ncIdentifiers) {
+						ncNames.add(id.getValue());
+					}
+					MetadataManager.INSTANCE.addNodegroup(mdTxnCtx,
+							new NodeGroup(ngName, ncNames));
+				}
+				break;
+			}
+			// drop statements
+			case DATAVERSE_DROP: {
+				DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+				String dvName = stmtDelete.getDataverseName().getValue();
+				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
+						ARTIFACT_KIND.DATAVERSE, dvName)) {
+					throw new AsterixException(
+							"Invalid Operation cannot drop dataverse " + dvName
+									+ " (protected by system)");
+				}
 
-                case CREATE_FUNCTION: {
-                    CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
-                    Function function = new Function(compiledDeclarations.getDataverseName(), cfs.getFunctionIdentifier().getFunctionName(),
-                            cfs.getFunctionIdentifier().getArity(), cfs.getParamList(), cfs.getFunctionBody());
-                    try {
-                        FunctionUtils.getFunctionDecl(function);
-                    } catch (Exception e) {
-                        throw new AsterixException("Unable to compile function definition", e);
-                    }
-                    MetadataManager.INSTANCE.addFunction(mdTxnCtx, new Function(
-                            compiledDeclarations.getDataverseName(), cfs.getFunctionIdentifier().getFunctionName(), cfs
-                                    .getFunctionIdentifier().getArity(), cfs.getParamList(), cfs.getFunctionBody()));
-                    break;
-                }
+				if (compiledDeclarations.isConnectedToDataverse())
+					compiledDeclarations.disconnectFromDataverse();
+				checkForDataverseConnection(false);
 
-                case FUNCTION_DROP: {
-                    checkForDataverseConnection(true);
-                    FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
-                    String functionName = stmtDropFunction.getFunctionName().getValue();
-                    FunctionIdentifier fId = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName,
-                            stmtDropFunction.getArity(), false);
-                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.FUNCTION, fId)) {
-                        throw new AsterixException("Invalid Operation cannot drop function " + functionName
-                                + " (protected by system)");
-                    }
-                    Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx,
-                            compiledDeclarations.getDataverseName(), functionName, stmtDropFunction.getArity());
-                    if (function == null) {
-                        if (!stmtDropFunction.getIfExists())
-                            throw new AlgebricksException("There is no function with this name " + functionName + ".");
-                    } else {
-                        MetadataManager.INSTANCE.dropFunction(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                                functionName, stmtDropFunction.getArity());
-                    }
-                    break;
-                }
-            }
-        }
+				Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
+						dvName);
+				if (dv == null) {
+					if (!stmtDelete.getIfExists()) {
+						throw new AlgebricksException(
+								"There is no dataverse with this name "
+										+ dvName + ".");
+					}
+				} else {
+					compiledDeclarations.connectToDataverse(dvName);
+					List<Dataset> datasets = MetadataManager.INSTANCE
+							.getDataverseDatasets(mdTxnCtx, dvName);
+					for (int j = 0; j < datasets.size(); j++) {
+						String datasetName = datasets.get(j).getDatasetName();
+						DatasetType dsType = datasets.get(j).getDatasetType();
+						if (dsType == DatasetType.INTERNAL
+								|| dsType == DatasetType.FEED) {
+							List<Index> indexes = MetadataManager.INSTANCE
+									.getDatasetIndexes(mdTxnCtx, dvName,
+											datasetName);
+							for (int k = 0; k < indexes.size(); k++) {
+								if (indexes.get(k).isSecondaryIndex()) {
+									compileIndexDropStatement(hcc, mdTxnCtx,
+											datasetName, indexes.get(k)
+													.getIndexName());
+								}
+							}
+						}
+						compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+					}
+					MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
+					if (compiledDeclarations.isConnectedToDataverse())
+						compiledDeclarations.disconnectFromDataverse();
+				}
+				break;
+			}
+			case DATASET_DROP: {
+				checkForDataverseConnection(true);
+				DropStatement stmtDelete = (DropStatement) stmt;
+				String datasetName = stmtDelete.getDatasetName().getValue();
+				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
+						ARTIFACT_KIND.DATASET, datasetName)) {
+					throw new AsterixException(
+							"Invalid Operation cannot drop dataset "
+									+ datasetName + " (protected by system)");
+				}
+				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), datasetName);
+				if (ds == null) {
+					if (!stmtDelete.getIfExists())
+						throw new AlgebricksException(
+								"There is no dataset with this name "
+										+ datasetName + ".");
+				} else {
+					if (ds.getDatasetType() == DatasetType.INTERNAL
+							|| ds.getDatasetType() == DatasetType.FEED) {
+						List<Index> indexes = MetadataManager.INSTANCE
+								.getDatasetIndexes(
+										mdTxnCtx,
+										compiledDeclarations.getDataverseName(),
+										datasetName);
+						for (int j = 0; j < indexes.size(); j++) {
+							if (indexes.get(j).isPrimaryIndex()) {
+								compileIndexDropStatement(hcc, mdTxnCtx,
+										datasetName, indexes.get(j)
+												.getIndexName());
+							}
+						}
+					}
+					compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+				}
+				break;
+			}
+			case INDEX_DROP: {
+				checkForDataverseConnection(true);
+				IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
+				String datasetName = stmtDelete.getDatasetName().getValue();
+				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), datasetName);
+				if (ds == null)
+					throw new AlgebricksException(
+							"There is no dataset with this name " + datasetName
+									+ ".");
+				if (ds.getDatasetType() == DatasetType.INTERNAL
+						|| ds.getDatasetType() == DatasetType.FEED) {
+					String indexName = stmtDelete.getIndexName().getValue();
+					Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
+							compiledDeclarations.getDataverseName(),
+							datasetName, indexName);
+					if (idx == null) {
+						if (!stmtDelete.getIfExists())
+							throw new AlgebricksException(
+									"There is no index with this name "
+											+ indexName + ".");
+					} else
+						compileIndexDropStatement(hcc, mdTxnCtx, datasetName,
+								indexName);
+				} else {
+					throw new AlgebricksException(
+							datasetName
+									+ " is an external dataset. Indexes are not maintained for external datasets.");
+				}
+				break;
+			}
+			case TYPE_DROP: {
+				checkForDataverseConnection(true);
+				TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
+				String typeName = stmtDelete.getTypeName().getValue();
+				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+						compiledDeclarations.getDataverseName(), typeName);
+				if (dt == null) {
+					if (!stmtDelete.getIfExists())
+						throw new AlgebricksException(
+								"There is no datatype with this name "
+										+ typeName + ".");
+				} else
+					MetadataManager.INSTANCE.dropDatatype(mdTxnCtx,
+							compiledDeclarations.getDataverseName(), typeName);
+				break;
+			}
+			case NODEGROUP_DROP: {
+				NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+				String nodegroupName = stmtDelete.getNodeGroupName().getValue();
+				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
+						ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
+					throw new AsterixException(
+							"Invalid Operation cannot drop nodegroup "
+									+ nodegroupName + " (protected by system)");
+				}
+				NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
+						nodegroupName);
+				if (ng == null) {
+					if (!stmtDelete.getIfExists())
+						throw new AlgebricksException(
+								"There is no nodegroup with this name "
+										+ nodegroupName + ".");
+				} else
+					MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx,
+							nodegroupName);
+				break;
+			}
 
-        if (disconnectFromDataverse) {
-            if (compiledDeclarations.isConnectedToDataverse()) {
-                compiledDeclarations.disconnectFromDataverse();
-            }
-        }
-    }
+			case CREATE_FUNCTION: {
+				CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+				Function function = new Function(
+						compiledDeclarations.getDataverseName(), cfs
+								.getFunctionIdentifier().getFunctionName(), cfs
+								.getFunctionIdentifier().getArity(),
+						cfs.getParamList(), cfs.getFunctionBody());
+				try {
+					FunctionUtils.getFunctionDecl(function);
+				} catch (Exception e) {
+					throw new AsterixException(
+							"Unable to compile function definition", e);
+				}
+				MetadataManager.INSTANCE
+						.addFunction(mdTxnCtx, new Function(
+								compiledDeclarations.getDataverseName(), cfs
+										.getFunctionIdentifier()
+										.getFunctionName(), cfs
+										.getFunctionIdentifier().getArity(),
+								cfs.getParamList(), cfs.getFunctionBody()));
+				break;
+			}
 
-    private void checkForDataverseConnection(boolean needConnection) throws AlgebricksException {
-        if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
-            if (needConnection)
-                throw new AlgebricksException("You need first to connect to a dataverse.");
-            else
-                throw new AlgebricksException("You need first to disconnect from the dataverse.");
-        }
-    }
+			case FUNCTION_DROP: {
+				checkForDataverseConnection(true);
+				FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+				String functionName = stmtDropFunction.getFunctionName()
+						.getValue();
+                    FunctionIdentifier fId = new FunctionIdentifier(
+						FunctionConstants.ASTERIX_NS, functionName,
+						stmtDropFunction.getArity());
+				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
+						ARTIFACT_KIND.FUNCTION, fId)) {
+					throw new AsterixException(
+							"Invalid Operation cannot drop function "
+									+ functionName + " (protected by system)");
+				}
+				Function function = MetadataManager.INSTANCE.getFunction(
+						mdTxnCtx, compiledDeclarations.getDataverseName(),
+						functionName, stmtDropFunction.getArity());
+				if (function == null) {
+					if (!stmtDropFunction.getIfExists())
+						throw new AlgebricksException(
+								"There is no function with this name "
+										+ functionName + ".");
+				} else {
+					MetadataManager.INSTANCE.dropFunction(mdTxnCtx,
+							compiledDeclarations.getDataverseName(),
+							functionName, stmtDropFunction.getArity());
+				}
+				break;
+			}
+			}
+		}
 
-    private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec) throws Exception {
-        System.out.println(jobSpec.toString());
-        executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
-    }
+		if (disconnectFromDataverse) {
+			if (compiledDeclarations.isConnectedToDataverse()) {
+				compiledDeclarations.disconnectFromDataverse();
+			}
+		}
+	}
 
-    public void executeJobArray(IHyracksClientConnection hcc, JobSpecification[] specs, PrintWriter out,
-            DisplayFormat pdf) throws Exception {
-        for (int i = 0; i < specs.length; i++) {
-            specs[i].setMaxReattempts(0);
-            JobId jobId = hcc.createJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
-            hcc.start(jobId);
-            hcc.waitForCompletion(jobId);
-        }
-    }
+	private void checkForDataverseConnection(boolean needConnection)
+			throws AlgebricksException {
+		if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
+			if (needConnection)
+				throw new AlgebricksException(
+						"You need first to connect to a dataverse.");
+			else
+				throw new AlgebricksException(
+						"You need first to disconnect from the dataverse.");
+		}
+	}
 
-    private void runCreateDatasetJob(IHyracksClientConnection hcc, String datasetName) throws AsterixException,
-            AlgebricksException, Exception {
-        runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName, compiledDeclarations));
-    }
-    
-    private void runCreateIndexJob(IHyracksClientConnection hcc, CreateIndexStatement stmtCreateIndex) throws Exception {
-        // TODO: Eventually CreateIndexStatement and CompiledCreateIndexStatement should be replaced by the corresponding metadata entity.
-        // For now we must still convert to a CompiledCreateIndexStatement here.
-        CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(stmtCreateIndex.getIndexName()
-                .getValue(), stmtCreateIndex.getDatasetName().getValue(), stmtCreateIndex.getFieldExprs(),
-                stmtCreateIndex.getIndexType());
-        JobSpecification spec = IndexOperations.buildSecondaryIndexCreationJobSpec(createIndexStmt,
-                compiledDeclarations);
-        if (spec == null) {
-            throw new AsterixException("Failed to create job spec for creating index '"
-                    + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
-        }
-        runJob(hcc, spec);
-    }
-	
-    private void compileDatasetDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
-            String datasetName) throws Exception {
-        CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(datasetName);
-        Dataset ds = MetadataManager.INSTANCE
-                .getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
-        if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
-            JobSpecification[] jobs = DatasetOperations.createDropDatasetJobSpec(cds, compiledDeclarations);
-            for (JobSpecification job : jobs)
-                runJob(hcc, job);
-        }
-        MetadataManager.INSTANCE.dropDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
-    }
+	private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec)
+			throws Exception {
+		System.out.println(jobSpec.toString());
+		executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
+	}
 
-    public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
-        return compiledDeclarations;
-    }
+	public void executeJobArray(IHyracksClientConnection hcc,
+			JobSpecification[] specs, PrintWriter out, DisplayFormat pdf)
+			throws Exception {
+		for (int i = 0; i < specs.length; i++) {
+			specs[i].setMaxReattempts(0);
+			JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
+			hcc.waitForCompletion(jobId);
+		}
+	}
+
+	private void runCreateDatasetJob(IHyracksClientConnection hcc,
+			String datasetName) throws AsterixException, AlgebricksException,
+			Exception {
+		runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName,
+				compiledDeclarations));
+	}
+
+	private void runCreateIndexJob(IHyracksClientConnection hcc,
+			CreateIndexStatement stmtCreateIndex) throws Exception {
+		// TODO: Eventually CreateIndexStatement and
+		// CompiledCreateIndexStatement should be replaced by the corresponding
+		// metadata entity.
+		// For now we must still convert to a CompiledCreateIndexStatement here.
+		CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(
+				stmtCreateIndex.getIndexName().getValue(), stmtCreateIndex
+						.getDatasetName().getValue(),
+				stmtCreateIndex.getFieldExprs(),
+				stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
+		JobSpecification spec = IndexOperations
+				.buildSecondaryIndexCreationJobSpec(createIndexStmt,
+						compiledDeclarations);
+		if (spec == null) {
+			throw new AsterixException(
+					"Failed to create job spec for creating index '"
+							+ stmtCreateIndex.getDatasetName() + "."
+							+ stmtCreateIndex.getIndexName() + "'");
+		}
+		runJob(hcc, spec);
+	}
+
+	private void compileDatasetDropStatement(IHyracksClientConnection hcc,
+			MetadataTransactionContext mdTxnCtx, String datasetName)
+			throws Exception {
+		CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(
+				datasetName);
+		Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
+				compiledDeclarations.getDataverseName(), datasetName);
+		if (ds.getDatasetType() == DatasetType.INTERNAL
+				|| ds.getDatasetType() == DatasetType.FEED) {
+			JobSpecification[] jobs = DatasetOperations
+					.createDropDatasetJobSpec(cds, compiledDeclarations);
+			for (JobSpecification job : jobs)
+				runJob(hcc, job);
+		}
+		MetadataManager.INSTANCE.dropDataset(mdTxnCtx,
+				compiledDeclarations.getDataverseName(), datasetName);
+	}
+
+	public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
+		return compiledDeclarations;
+	}
 
-    private void compileIndexDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
-            String datasetName, String indexName) throws Exception {
-        CompiledIndexDropStatement cds = new CompiledIndexDropStatement(datasetName, indexName);
-        runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, compiledDeclarations));
-        MetadataManager.INSTANCE.dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName, indexName);
-    }
+	private void compileIndexDropStatement(IHyracksClientConnection hcc,
+			MetadataTransactionContext mdTxnCtx, String datasetName,
+			String indexName) throws Exception {
+		CompiledIndexDropStatement cds = new CompiledIndexDropStatement(
+				datasetName, indexName);
+		runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds,
+				compiledDeclarations));
+		MetadataManager.INSTANCE
+				.dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
+						datasetName, indexName);
+	}
 
-    private Map<String, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
-            throws AlgebricksException, MetadataException {
-        Map<String, IAType> typeMap = new HashMap<String, IAType>();
-        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
-        Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
-        Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
+	private Map<String, IAType> computeTypes(
+			MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
+			throws AlgebricksException, MetadataException {
+		Map<String, IAType> typeMap = new HashMap<String, IAType>();
+		Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+		Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
+		Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
 
-        firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
-        secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
+		firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes,
+				incompleteTopLevelTypeReferences);
+		secondPass(mdTxnCtx, typeMap, incompleteFieldTypes,
+				incompleteItemTypes, incompleteTopLevelTypeReferences);
 
-        return typeMap;
-    }
+		return typeMap;
+	}
 
-    private void secondPass(MetadataTransactionContext mdTxnCtx, Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException, MetadataException {
-        // solve remaining top level references
-        for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
-            IAType t;// = typeMap.get(trefName);
-            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                    trefName);
-            if (dt == null) {
-                throw new AlgebricksException("Could not resolve type " + trefName);
-            } else
-                t = dt.getDatatype();
-            for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
-                typeMap.put(tname, t);
-            }
-        }
-        // solve remaining field type references
-        for (String trefName : incompleteFieldTypes.keySet()) {
-            IAType t;// = typeMap.get(trefName);
-            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                    trefName);
-            if (dt == null) {
-                throw new AlgebricksException("Could not resolve type " + trefName);
-            } else
-                t = dt.getDatatype();
-            Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
-            for (ARecordType recType : fieldsToFix.keySet()) {
-                List<Integer> positions = fieldsToFix.get(recType);
-                IAType[] fldTypes = recType.getFieldTypes();
-                for (Integer pos : positions) {
-                    if (fldTypes[pos] == null) {
-                        fldTypes[pos] = t;
-                    } else { // nullable
-                        AUnionType nullableUnion = (AUnionType) fldTypes[pos];
-                        nullableUnion.setTypeAtIndex(t, 1);
-                    }
-                }
-            }
-        }
-        // solve remaining item type references
-        for (String trefName : incompleteItemTypes.keySet()) {
-            IAType t;// = typeMap.get(trefName);
-            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
-                    trefName);
-            if (dt == null) {
-                throw new AlgebricksException("Could not resolve type " + trefName);
-            } else
-                t = dt.getDatatype();
-            for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
-                act.setItemType(t);
-            }
-        }
-    }
+	private void secondPass(MetadataTransactionContext mdTxnCtx,
+			Map<String, IAType> typeMap,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+			Map<String, List<String>> incompleteTopLevelTypeReferences)
+			throws AlgebricksException, MetadataException {
+		// solve remaining top level references
+		for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
+			IAType t;// = typeMap.get(trefName);
+			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+					compiledDeclarations.getDataverseName(), trefName);
+			if (dt == null) {
+				throw new AlgebricksException("Could not resolve type "
+						+ trefName);
+			} else
+				t = dt.getDatatype();
+			for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
+				typeMap.put(tname, t);
+			}
+		}
+		// solve remaining field type references
+		for (String trefName : incompleteFieldTypes.keySet()) {
+			IAType t;// = typeMap.get(trefName);
+			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+					compiledDeclarations.getDataverseName(), trefName);
+			if (dt == null) {
+				throw new AlgebricksException("Could not resolve type "
+						+ trefName);
+			} else
+				t = dt.getDatatype();
+			Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes
+					.get(trefName);
+			for (ARecordType recType : fieldsToFix.keySet()) {
+				List<Integer> positions = fieldsToFix.get(recType);
+				IAType[] fldTypes = recType.getFieldTypes();
+				for (Integer pos : positions) {
+					if (fldTypes[pos] == null) {
+						fldTypes[pos] = t;
+					} else { // nullable
+						AUnionType nullableUnion = (AUnionType) fldTypes[pos];
+						nullableUnion.setTypeAtIndex(t, 1);
+					}
+				}
+			}
+		}
+		// solve remaining item type references
+		for (String trefName : incompleteItemTypes.keySet()) {
+			IAType t;// = typeMap.get(trefName);
+			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+					compiledDeclarations.getDataverseName(), trefName);
+			if (dt == null) {
+				throw new AlgebricksException("Could not resolve type "
+						+ trefName);
+			} else
+				t = dt.getDatatype();
+			for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
+				act.setItemType(t);
+			}
+		}
+	}
 
-    private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException {
+	private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+			Map<String, List<String>> incompleteTopLevelTypeReferences)
+			throws AlgebricksException {
 
-        TypeExpression texpr = td.getTypeDef();
-        String tdname = td.getIdent().getValue();
-        if (builtinTypeMap.get(tdname) != null) {
-            throw new AlgebricksException("Cannot redefine builtin type " + tdname + " .");
-        }
-        switch (texpr.getTypeKind()) {
-            case TYPEREFERENCE: {
-                TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-                IAType t = solveTypeReference(tre, typeMap);
-                if (t != null) {
-                    typeMap.put(tdname, t);
-                } else {
-                    addIncompleteTopLevelTypeReference(tdname, tre, incompleteTopLevelTypeReferences);
-                }
-                break;
-            }
-            case RECORD: {
-                RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
-                ARecordType recType = computeRecordType(tdname, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
-                typeMap.put(tdname, recType);
-                break;
-            }
-            case ORDEREDLIST: {
-                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-                AOrderedListType olType = computeOrderedListType(tdname, oltd, typeMap, incompleteItemTypes,
-                        incompleteFieldTypes);
-                typeMap.put(tdname, olType);
-                break;
-            }
-            case UNORDEREDLIST: {
-                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-                AUnorderedListType ulType = computeUnorderedListType(tdname, ultd, typeMap, incompleteItemTypes,
-                        incompleteFieldTypes);
-                typeMap.put(tdname, ulType);
-                break;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-    }
+		TypeExpression texpr = td.getTypeDef();
+		String tdname = td.getIdent().getValue();
+		if (builtinTypeMap.get(tdname) != null) {
+			throw new AlgebricksException("Cannot redefine builtin type "
+					+ tdname + " .");
+		}
+		switch (texpr.getTypeKind()) {
+		case TYPEREFERENCE: {
+			TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+			IAType t = solveTypeReference(tre, typeMap);
+			if (t != null) {
+				typeMap.put(tdname, t);
+			} else {
+				addIncompleteTopLevelTypeReference(tdname, tre,
+						incompleteTopLevelTypeReferences);
+			}
+			break;
+		}
+		case RECORD: {
+			RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
+			ARecordType recType = computeRecordType(tdname, rtd, typeMap,
+					incompleteFieldTypes, incompleteItemTypes);
+			typeMap.put(tdname, recType);
+			break;
+		}
+		case ORDEREDLIST: {
+			OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+			AOrderedListType olType = computeOrderedListType(tdname, oltd,
+					typeMap, incompleteItemTypes, incompleteFieldTypes);
+			typeMap.put(tdname, olType);
+			break;
+		}
+		case UNORDEREDLIST: {
+			UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+			AUnorderedListType ulType = computeUnorderedListType(tdname, ultd,
+					typeMap, incompleteItemTypes, incompleteFieldTypes);
+			typeMap.put(tdname, ulType);
+			break;
+		}
+		default: {
+			throw new IllegalStateException();
+		}
+		}
+	}
 
-    private AOrderedListType computeOrderedListType(String typeName, OrderedListTypeDefinition oltd,
-            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        TypeExpression tExpr = oltd.getItemTypeExpression();
-        AOrderedListType aolt = new AOrderedListType(null, typeName);
-        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt);
-        return aolt;
-    }
+	private AOrderedListType computeOrderedListType(String typeName,
+			OrderedListTypeDefinition oltd, Map<String, IAType> typeMap,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+		TypeExpression tExpr = oltd.getItemTypeExpression();
+		AOrderedListType aolt = new AOrderedListType(null, typeName);
+		setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
+				incompleteFieldTypes, aolt);
+		return aolt;
+	}
 
-    private AUnorderedListType computeUnorderedListType(String typeName, UnorderedListTypeDefinition ultd,
-            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        TypeExpression tExpr = ultd.getItemTypeExpression();
-        AUnorderedListType ault = new AUnorderedListType(null, typeName);
-        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault);
-        return ault;
-    }
+	private AUnorderedListType computeUnorderedListType(String typeName,
+			UnorderedListTypeDefinition ultd, Map<String, IAType> typeMap,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+		TypeExpression tExpr = ultd.getItemTypeExpression();
+		AUnorderedListType ault = new AUnorderedListType(null, typeName);
+		setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
+				incompleteFieldTypes, ault);
+		return ault;
+	}
 
-    private void setCollectionItemType(TypeExpression tExpr, Map<String, IAType> typeMap,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act) {
-        switch (tExpr.getTypeKind()) {
-            case ORDEREDLIST: {
-                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
-                IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                act.setItemType(t);
-                break;
-            }
-            case UNORDEREDLIST: {
-                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
-                IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                act.setItemType(t);
-                break;
-            }
-            case RECORD: {
-                RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
-                IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
-                act.setItemType(t);
-                break;
-            }
-            case TYPEREFERENCE: {
-                TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
-                IAType tref = solveTypeReference(tre, typeMap);
-                if (tref != null) {
-                    act.setItemType(tref);
-                } else {
-                    addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes);
-                }
-                break;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-    }
+	private void setCollectionItemType(TypeExpression tExpr,
+			Map<String, IAType> typeMap,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+			AbstractCollectionType act) {
+		switch (tExpr.getTypeKind()) {
+		case ORDEREDLIST: {
+			OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
+			IAType t = computeOrderedListType(null, oltd, typeMap,
+					incompleteItemTypes, incompleteFieldTypes);
+			act.setItemType(t);
+			break;
+		}
+		case UNORDEREDLIST: {
+			UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
+			IAType t = computeUnorderedListType(null, ultd, typeMap,
+					incompleteItemTypes, incompleteFieldTypes);
+			act.setItemType(t);
+			break;
+		}
+		case RECORD: {
+			RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
+			IAType t = computeRecordType(null, rtd, typeMap,
+					incompleteFieldTypes, incompleteItemTypes);
+			act.setItemType(t);
+			break;
+		}
+		case TYPEREFERENCE: {
+			TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
+			IAType tref = solveTypeReference(tre, typeMap);
+			if (tref != null) {
+				act.setItemType(tref);
+			} else {
+				addIncompleteCollectionTypeReference(act, tre,
+						incompleteItemTypes);
+			}
+			break;
+		}
+		default: {
+			throw new IllegalStateException();
+		}
+		}
+	}
 
-    private ARecordType computeRecordType(String typeName, RecordTypeDefinition rtd, Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-        List<String> names = rtd.getFieldNames();
-        int n = names.size();
-        String[] fldNames = new String[n];
-        IAType[] fldTypes = new IAType[n];
-        int i = 0;
-        for (String s : names) {
-            fldNames[i++] = s;
-        }
-        boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
-        ARecordType recType = new ARecordType(typeName, fldNames, fldTypes, isOpen);
-        for (int j = 0; j < n; j++) {
-            TypeExpression texpr = rtd.getFieldTypes().get(j);
-            switch (texpr.getTypeKind()) {
-                case TYPEREFERENCE: {
-                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-                    IAType tref = solveTypeReference(tre, typeMap);
-                    if (tref != null) {
-                        if (!rtd.getNullableFields().get(j)) { // not nullable
-                            fldTypes[j] = tref;
-                        } else { // nullable
-                            fldTypes[j] = makeUnionWithNull(null, tref);
-                        }
-                    } else {
-                        addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
-                        if (rtd.getNullableFields().get(j)) {
-                            fldTypes[j] = makeUnionWithNull(null, null);
-                        }
-                    }
-                    break;
-                }
-                case RECORD: {
-                    RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
-                    IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes);
-                    if (!rtd.getNullableFields().get(j)) { // not nullable
-                        fldTypes[j] = t2;
-                    } else { // nullable
-                        fldTypes[j] = makeUnionWithNull(null, t2);
-                    }
-                    break;
-                }
-                case ORDEREDLIST: {
-                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-                    IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
-                    break;
-                }
-                case UNORDEREDLIST: {
-                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-                    IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
-                    break;
-                }
-                default: {
-                    throw new IllegalStateException();
-                }
-            }
+	private ARecordType computeRecordType(String typeName,
+			RecordTypeDefinition rtd, Map<String, IAType> typeMap,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+		List<String> names = rtd.getFieldNames();
+		int n = names.size();
+		String[] fldNames = new String[n];
+		IAType[] fldTypes = new IAType[n];
+		int i = 0;
+		for (String s : names) {
+			fldNames[i++] = s;
+		}
+		boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
+		ARecordType recType = new ARecordType(typeName, fldNames, fldTypes,
+				isOpen);
+		for (int j = 0; j < n; j++) {
+			TypeExpression texpr = rtd.getFieldTypes().get(j);
+			switch (texpr.getTypeKind()) {
+			case TYPEREFERENCE: {
+				TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+				IAType tref = solveTypeReference(tre, typeMap);
+				if (tref != null) {
+					if (!rtd.getNullableFields().get(j)) { // not nullable
+						fldTypes[j] = tref;
+					} else { // nullable
+						fldTypes[j] = makeUnionWithNull(null, tref);
+					}
+				} else {
+					addIncompleteFieldTypeReference(recType, j, tre,
+							incompleteFieldTypes);
+					if (rtd.getNullableFields().get(j)) {
+						fldTypes[j] = makeUnionWithNull(null, null);
+					}
+				}
+				break;
+			}
+			case RECORD: {
+				RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
+				IAType t2 = computeRecordType(null, recTypeDef2, typeMap,
+						incompleteFieldTypes, incompleteItemTypes);
+				if (!rtd.getNullableFields().get(j)) { // not nullable
+					fldTypes[j] = t2;
+				} else { // nullable
+					fldTypes[j] = makeUnionWithNull(null, t2);
+				}
+				break;
+			}
+			case ORDEREDLIST: {
+				OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+				IAType t2 = computeOrderedListType(null, oltd, typeMap,
+						incompleteItemTypes, incompleteFieldTypes);
+				fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
+						null, t2) : t2;
+				break;
+			}
+			case UNORDEREDLIST: {
+				UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+				IAType t2 = computeUnorderedListType(null, ultd, typeMap,
+						incompleteItemTypes, incompleteFieldTypes);
+				fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
+						null, t2) : t2;
+				break;
+			}
+			default: {
+				throw new IllegalStateException();
+			}
+			}
 
-        }
+		}
 
-        return recType;
-    }
+		return recType;
+	}
 
-    private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
-        ArrayList<IAType> unionList = new ArrayList<IAType>(2);
-        unionList.add(BuiltinType.ANULL);
-        unionList.add(type);
-        return new AUnionType(unionList, unionTypeName);
-    }
+	private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
+		ArrayList<IAType> unionList = new ArrayList<IAType>(2);
+		unionList.add(BuiltinType.ANULL);
+		unionList.add(type);
+		return new AUnionType(unionList, unionTypeName);
+	}
 
-    private void addIncompleteCollectionTypeReference(AbstractCollectionType collType, TypeReferenceExpression tre,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-        String typeName = tre.getIdent().getValue();
-        List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeName);
-        if (typeList == null) {
-            typeList = new LinkedList<AbstractCollectionType>();
-            incompleteItemTypes.put(typeName, typeList);
-        }
-        typeList.add(collType);
-    }
+	private void addIncompleteCollectionTypeReference(
+			AbstractCollectionType collType, TypeReferenceExpression tre,
+			Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+		String typeName = tre.getIdent().getValue();
+		List<AbstractCollectionType> typeList = incompleteItemTypes
+				.get(typeName);
+		if (typeList == null) {
+			typeList = new LinkedList<AbstractCollectionType>();
+			incompleteItemTypes.put(typeName, typeList);
+		}
+		typeList.add(collType);
+	}
 
-    private void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition, TypeReferenceExpression tre,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        String typeName = tre.getIdent().getValue();
-        Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
-        if (refMap == null) {
-            refMap = new HashMap<ARecordType, List<Integer>>();
-            incompleteFieldTypes.put(typeName, refMap);
-        }
-        List<Integer> typeList = refMap.get(recType);
-        if (typeList == null) {
-            typeList = new ArrayList<Integer>();
-            refMap.put(recType, typeList);
-        }
-        typeList.add(fldPosition);
-    }
+	private void addIncompleteFieldTypeReference(ARecordType recType,
+			int fldPosition, TypeReferenceExpression tre,
+			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+		String typeName = tre.getIdent().getValue();
+		Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes
+				.get(typeName);
+		if (refMap == null) {
+			refMap = new HashMap<ARecordType, List<Integer>>();
+			incompleteFieldTypes.put(typeName, refMap);
+		}
+		List<Integer> typeList = refMap.get(recType);
+		if (typeList == null) {
+			typeList = new ArrayList<Integer>();
+			refMap.put(recType, typeList);
+		}
+		typeList.add(fldPosition);
+	}
 
-    private void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) {
-        String name = tre.getIdent().getValue();
-        List<String> refList = incompleteTopLevelTypeReferences.get(name);
-        if (refList == null) {
-            refList = new LinkedList<String>();
-            incompleteTopLevelTypeReferences.put(name, refList);
-        }
-        refList.add(tdeclName);
-    }
+	private void addIncompleteTopLevelTypeReference(String tdeclName,
+			TypeReferenceExpression tre,
+			Map<String, List<String>> incompleteTopLevelTypeReferences) {
+		String name = tre.getIdent().getValue();
+		List<String> refList = incompleteTopLevelTypeReferences.get(name);
+		if (refList == null) {
+			refList = new LinkedList<String>();
+			incompleteTopLevelTypeReferences.put(name, refList);
+		}
+		refList.add(tdeclName);
+	}
 
-    private IAType solveTypeReference(TypeReferenceExpression tre, Map<String, IAType> typeMap) {
-        String name = tre.getIdent().getValue();
-        IAType builtin = builtinTypeMap.get(name);
-        if (builtin != null) {
-            return builtin;
-        } else {
-            return typeMap.get(name);
-        }
-    }
+	private IAType solveTypeReference(TypeReferenceExpression tre,
+			Map<String, IAType> typeMap) {
+		String name = tre.getIdent().getValue();
+		IAType builtin = builtinTypeMap.get(name);
+		if (builtin != null) {
+			return builtin;
+		} else {
+			return typeMap.get(name);
+		}
+	}
 
-    public static interface ICompiledStatement {
+	public static interface ICompiledStatement {
 
-        public abstract Kind getKind();
-    }
+		public abstract Kind getKind();
+	}
 
-    public static class CompiledLoadFromFileStatement implements ICompiledStatement, IParseFileSplitsDecl {
-        private String datasetName;
-        private FileSplit[] splits;
-        private boolean alreadySorted;
-        private Character delimChar;
+	public static class CompiledLoadFromFileStatement implements
+			ICompiledStatement, IParseFileSplitsDecl {
+		private String datasetName;
+		private FileSplit[] splits;
+		private boolean alreadySorted;
+		private Character delimChar;
 
-        public CompiledLoadFromFileStatement(String datasetName, FileSplit[] splits, Character delimChar,
-                boolean alreadySorted) {
-            this.datasetName = datasetName;
-            this.splits = splits;
-            this.delimChar = delimChar;
-            this.alreadySorted = alreadySorted;
-        }
+		public CompiledLoadFromFileStatement(String datasetName,
+				FileSplit[] splits, Character delimChar, boolean alreadySorted) {
+			this.datasetName = datasetName;
+			this.splits = splits;
+			this.delimChar = delimChar;
+			this.alreadySorted = alreadySorted;
+		}
 
-        public String getDatasetName() {
-            return datasetName;
-        }
+		public String getDatasetName() {
+			return datasetName;
+		}
 
-        @Override
-        public FileSplit[] getSplits() {
-            return splits;
-        }
+		@Override
+		public FileSplit[] getSplits() {
+			return splits;
+		}
 
-        @Override
-        public Character getDelimChar() {
-            return delimChar;
-        }
+		@Override
+		public Character getDelimChar() {
+			return delimChar;
+		}
 
-        public boolean alreadySorted() {
-            return alreadySorted;
-        }
+		public boolean alreadySorted() {
+			return alreadySorted;
+		}
 
-        @Override
-        public boolean isDelimitedFileFormat() {
-            return delimChar != null;
-        }
+		@Override
+		public boolean isDelimitedFileFormat() {
+			return delimChar != null;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.LOAD_FROM_FILE;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.LOAD_FROM_FILE;
+		}
+	}
 
-    public static class CompiledWriteFromQueryResultStatement implements ICompiledStatement {
+	public static class CompiledWriteFromQueryResultStatement implements
+			ICompiledStatement {
 
-        private String datasetName;
-        private Query query;
-        private int varCounter;
+		private String datasetName;
+		private Query query;
+		private int varCounter;
 
-        public CompiledWriteFromQueryResultStatement(String datasetName, Query query, int varCounter) {
-            this.datasetName = datasetName;
-            this.query = query;
-            this.varCounter = varCounter;
-        }
+		public CompiledWriteFromQueryResultStatement(String datasetName,
+				Query query, int varCounter) {
+			this.datasetName = datasetName;
+			this.query = query;
+			this.varCounter = varCounter;
+		}
 
-        public String getDatasetName() {
-            return datasetName;
-        }
+		public String getDatasetName() {
+			return datasetName;
+		}
 
-        public int getVarCounter() {
-            return varCounter;
-        }
+		public int getVarCounter() {
+			return varCounter;
+		}
 
-        public Query getQuery() {
-            return query;
-        }
+		public Query getQuery() {
+			return query;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.WRITE_FROM_QUERY_RESULT;
-        }
+		@Override
+		public Kind getKind() {
+			return Kind.WRITE_FROM_QUERY_RESULT;
+		}
 
-    }
+	}
 
-    public static class CompiledDatasetDropStatement implements ICompiledStatement {
-        private String datasetName;
+	public static class CompiledDatasetDropStatement implements
+			ICompiledStatement {
+		private String datasetName;
 
-        public CompiledDatasetDropStatement(String datasetName) {
-            this.datasetName = datasetName;
-        }
+		public CompiledDatasetDropStatement(String datasetName) {
+			this.datasetName = datasetName;
+		}
 
-        public String getDatasetName() {
-            return datasetName;
-        }
+		public String getDatasetName() {
+			return datasetName;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.DATASET_DROP;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.DATASET_DROP;
+		}
+	}
 
-    // added by yasser
-    public static class CompiledCreateDataverseStatement implements ICompiledStatement {
-        private String dataverseName;
-        private String format;
+	// added by yasser
+	public static class CompiledCreateDataverseStatement implements
+			ICompiledStatement {
+		private String dataverseName;
+		private String format;
 
-        public CompiledCreateDataverseStatement(String dataverseName, String format) {
-            this.dataverseName = dataverseName;
-            this.format = format;
-        }
+		public CompiledCreateDataverseStatement(String dataverseName,
+				String format) {
+			this.dataverseName = dataverseName;
+			this.format = format;
+		}
 
-        public String getDataverseName() {
-            return dataverseName;
-        }
+		public String getDataverseName() {
+			return dataverseName;
+		}
 
-        public String getFormat() {
-            return format;
-        }
+		public String getFormat() {
+			return format;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.CREATE_DATAVERSE;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.CREATE_DATAVERSE;
+		}
+	}
 
-    public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
-        private String nodeGroupName;
+	public static class CompiledNodeGroupDropStatement implements
+			ICompiledStatement {
+		private String nodeGroupName;
 
-        public CompiledNodeGroupDropStatement(String nodeGroupName) {
-            this.nodeGroupName = nodeGroupName;
-        }
+		public CompiledNodeGroupDropStatement(String nodeGroupName) {
+			this.nodeGroupName = nodeGroupName;
+		}
 
-        public String getNodeGroupName() {
-            return nodeGroupName;
-        }
+		public String getNodeGroupName() {
+			return nodeGroupName;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.NODEGROUP_DROP;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.NODEGROUP_DROP;
+		}
+	}
 
-    public static class CompiledIndexDropStatement implements ICompiledStatement {
-        private String datasetName;
-        private String indexName;
+	public static class CompiledIndexDropStatement implements
+			ICompiledStatement {
+		private String datasetName;
+		private String indexName;
 
-        public CompiledIndexDropStatement(String datasetName, String indexName) {
-            this.datasetName = datasetName;
-            this.indexName = indexName;
-        }
+		public CompiledIndexDropStatement(String datasetName, String indexName) {
+			this.datasetName = datasetName;
+			this.indexName = indexName;
+		}
 
-        public String getDatasetName() {
-            return datasetName;
-        }
+		public String getDatasetName() {
+			return datasetName;
+		}
 
-        public String getIndexName() {
-            return indexName;
-        }
+		public String getIndexName() {
+			return indexName;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.INDEX_DROP;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.INDEX_DROP;
+		}
+	}
 
-    public static class CompiledDataverseDropStatement implements ICompiledStatement {
-        private String dataverseName;
-        private boolean ifExists;
+	public static class CompiledDataverseDropStatement implements
+			ICompiledStatement {
+		private String dataverseName;
+		private boolean ifExists;
 
-        public CompiledDataverseDropStatement(String dataverseName, boolean ifExists) {
-            this.dataverseName = dataverseName;
-            this.ifExists = ifExists;
-        }
+		public CompiledDataverseDropStatement(String dataverseName,
+				boolean ifExists) {
+			this.dataverseName = dataverseName;
+			this.ifExists = ifExists;
+		}
 
-        public String getDataverseName() {
-            return dataverseName;
-        }
+		public String getDataverseName() {
+			return dataverseName;
+		}
 
-        public boolean getIfExists() {
-            return ifExists;
-        }
+		public boolean getIfExists() {
+			return ifExists;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.DATAVERSE_DROP;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.DATAVERSE_DROP;
+		}
+	}
 
-    public static class CompiledTypeDropStatement implements ICompiledStatement {
-        private String typeName;
+	public static class CompiledTypeDropStatement implements ICompiledStatement {
+		private String typeName;
 
-        public CompiledTypeDropStatement(String nodeGroupName) {
-            this.typeName = nodeGroupName;
-        }
+		public CompiledTypeDropStatement(String nodeGroupName) {
+			this.typeName = nodeGroupName;
+		}
 
-        public String getTypeName() {
-            return typeName;
-        }
+		public String getTypeName() {
+			return typeName;
+		}
 
-        @Override
-        public Kind getKind() {
-            return Kind.TYPE_DROP;
-        }
-    }
+		@Override
+		public Kind getKind() {
+			return Kind.TYPE_DROP;
+		}
+	}
 }
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 5aa0f6e..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
@@ -27,11 +27,11 @@
 import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -95,32 +95,38 @@
         IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
 
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
         }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return new JobSpecification[0];
         }
 
-        List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
-
+        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        int numSecondaryIndexes = 0;
+        for (Index index : datasetIndexes) {
+            if (index.isSecondaryIndex()) {
+                numSecondaryIndexes++;
+            }
+        }
         JobSpecification[] specs;
-
-        if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
-            int n = secondaryIndexes.size();
-            specs = new JobSpecification[n + 1];
+        if (numSecondaryIndexes > 0) {
+            specs = new JobSpecification[numSecondaryIndexes + 1];
             int i = 0;
             // First, drop secondary indexes.
-            for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-                specs[i] = new JobSpecification();
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
-                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, acid.getIndexName());
-                TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
-                        storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
-                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
-                        idxSplitsAndConstraint.second);
-                i++;
+            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++;
+                }
             }
         } else {
             specs = new JobSpecification[1];
@@ -140,17 +146,17 @@
         return specs;
     }
 
-    // TODO: Lots of common code in this file. Refactor everything after merging in asterix-fix-issue-9.
     public static JobSpecification createDatasetJobSpec(String datasetName, AqlCompiledMetadataDeclarations metadata)
             throws AsterixException, AlgebricksException {
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName);
         }
+        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         JobSpecification spec = new JobSpecification();
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
         FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
@@ -170,49 +176,47 @@
         return spec;
     }
 
+    @SuppressWarnings("rawtypes")
     public static Job createLoadDatasetJobSpec(CompiledLoadFromFileStatement loadStmt,
             AqlCompiledMetadataDeclarations metadata) throws AsterixException, AlgebricksException {
         String datasetName = loadStmt.getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName);
         }
-        if (compiledDatasetDecl.getDatasetType() != DatasetType.INTERNAL
-                && compiledDatasetDecl.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             throw new AsterixException("Cannot load data into dataset  (" + datasetName + ")" + "of type "
-                    + compiledDatasetDecl.getDatasetType());
+                    + dataset.getDatasetType());
         }
         JobSpecification spec = new JobSpecification();
 
-        ARecordType itemType = (ARecordType) metadata.findType(compiledDatasetDecl.getItemTypeName());
+        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         IDataFormat format = metadata.getFormat();
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
 
-        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryHashFunctionFactoryProvider());
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(dataset, itemType,
+                metadata.getFormat().getBinaryHashFunctionFactoryProvider());
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
 
-        AqlCompiledExternalDatasetDetails externalDatasetDetails = new AqlCompiledExternalDatasetDetails(
-                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;
         AlgebricksPartitionConstraint scannerPc = p.second;
-        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(compiledDatasetDecl, payloadSerde,
+        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde,
                 metadata.getFormat());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, scanner, scannerPc);
 
-        AssignRuntimeFactory assign = makeAssignRuntimeFactory(compiledDatasetDecl);
+        AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, metadata.getFormat());
         AlgebricksMetaOperatorDescriptor asterixOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { recDesc });
 
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixOp, scannerPc);
 
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+        int numKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         int[] keys = new int[numKeys];
         for (int i = 0; i < numKeys; i++) {
             keys[i] = i + 1;
@@ -270,14 +274,14 @@
         return fs.getNodeName() + ":" + fs.getLocalFile().toString();
     }
 
-    private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+    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 = partitioningFunctions
-                    .get(i);
+            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
+                    .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
             evalFactories[i] = evalFactoryAndType.first;
         }
         int[] outColumns = new int[numKeys];
@@ -296,21 +300,18 @@
         return new AssignRuntimeFactory(outColumns, sefs, projectionList);
     }
 
-    private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
+    @SuppressWarnings("rawtypes")
+    private static RecordDescriptor computePayloadKeyRecordDescriptor(Dataset dataset, ARecordType itemType,
             ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
         ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
         recordFields[0] = payloadSerde;
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
             recordFields[i + 1] = keySerde;
         }
         return new RecordDescriptor(recordFields);
     }
-
 }
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 4d7cbe2..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
@@ -24,30 +24,18 @@
 import edu.uci.ics.asterix.feed.comm.FeedMessage;
 import edu.uci.ics.asterix.feed.comm.IFeedMessage;
 import edu.uci.ics.asterix.feed.comm.IFeedMessage.MessageType;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledControlFeedStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 
 public class FeedOperations {
@@ -77,12 +65,17 @@
 
         LOGGER.info(" DATASETPATH: " + datasetPath);
 
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset;
+        try {
+            dataset = metadata.findDataset(datasetName);
+        } catch (AlgebricksException e) {
+            throw new AsterixException(e);
+        }
+        if (dataset == null) {
             throw new AsterixException("FEED DATASET: No metadata for dataset " + datasetName);
         }
-        if (adecl.getDatasetType() != DatasetType.FEED) {
-            throw new AsterixException("Operation not support for dataset type  " + adecl.getDatasetType());
+        if (dataset.getDatasetType() != DatasetType.FEED) {
+            throw new AsterixException("Operation not support for dataset type  " + dataset.getDatasetType());
         }
 
         JobSpecification spec = new JobSpecification();
@@ -107,8 +100,8 @@
 
         try {
             Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider.buildFeedMessengerRuntime(
-                    spec, metadata, (AqlCompiledFeedDatasetDetails) adecl.getAqlCompiledDatasetDetails(),
-                    metadata.getDataverseName(), datasetName, feedMessages);
+                    spec, metadata, (FeedDatasetDetails) dataset.getDatasetDetails(), metadata.getDataverseName(),
+                    datasetName, feedMessages);
             feedMessenger = p.first;
             messengerPc = p.second;
         } catch (AlgebricksException e) {
@@ -126,54 +119,4 @@
         return spec;
 
     }
-
-    private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
-
-        int index = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            evalFactories[index++] = evalFactoryAndType.first;
-        }
-
-        int[] outColumns = new int[numKeys];
-        int[] projectionList = new int[numKeys + 1];
-        projectionList[0] = 0;
-
-        for (int i = 0; i < numKeys; i++) {
-            outColumns[i] = i + 1;
-            projectionList[i + 1] = i + 1;
-        }
-        IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[evalFactories.length];
-        for (int i = 0; i < evalFactories.length; ++i) {
-            sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
-                    evalFactories[i]);
-        }
-        return new AssignRuntimeFactory(outColumns, sefs, projectionList);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
-            ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
-
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
-        recordFields[0] = payloadSerde;
-        int index = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
-            recordFields[index + 1] = keySerde;
-            index++;
-        }
-        return new RecordDescriptor(recordFields);
-    }
-
-    private static String stringOf(FileSplit fs) {
-        return fs.getNodeName() + ":" + fs.getLocalFile().toString();
-    }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 7e8ede1..7bd7ae6 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.file;
 
-
 import edu.uci.ics.asterix.aql.translator.DdlTranslator.CompiledIndexDropStatement;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
 import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
@@ -28,16 +27,18 @@
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(CompiledCreateIndexStatement createIndexStmt,
             AqlCompiledMetadataDeclarations metadata) throws AsterixException, AlgebricksException {
-        SecondaryIndexCreator secondaryIndexCreator = SecondaryIndexCreator.createIndexCreator(createIndexStmt, metadata, physicalOptimizationConfig);
+        SecondaryIndexCreator secondaryIndexCreator = SecondaryIndexCreator.createIndexCreator(createIndexStmt,
+                metadata, physicalOptimizationConfig);
         return secondaryIndexCreator.buildCreationJobSpec();
     }
-    
+
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(CompiledCreateIndexStatement createIndexStmt,
             AqlCompiledMetadataDeclarations metadata) throws AsterixException, AlgebricksException {
-        SecondaryIndexCreator secondaryIndexCreator = SecondaryIndexCreator.createIndexCreator(createIndexStmt, metadata, physicalOptimizationConfig);
+        SecondaryIndexCreator secondaryIndexCreator = SecondaryIndexCreator.createIndexCreator(createIndexStmt,
+                metadata, physicalOptimizationConfig);
         return secondaryIndexCreator.buildLoadingJobSpec();
     }
-    
+
     public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement deleteStmt,
             AqlCompiledMetadataDeclarations datasetDecls) throws AlgebricksException, MetadataException {
         String datasetName = deleteStmt.getDatasetName();
@@ -57,4 +58,4 @@
 
         return spec;
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index 83d7886..b9ad908 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
 
 public class SecondaryBTreeCreator extends SecondaryIndexCreator {
-    
+
     protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf) {
         super(physOptConf);
     }
@@ -38,29 +38,28 @@
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         return spec;
     }
-    
+
     @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
-        
+
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
-        
+
         // Create primary index scan op.
         BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
-        
+
         // Assign op.
         AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
-        
+
         // If any of the secondary fields are nullable, then add a select op that filters nulls.
         AlgebricksMetaOperatorDescriptor selectOp = null;
         if (anySecondaryKeyIsNullable) {
             selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
         }
-        
+
         // Sort by secondary keys.
         ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
 
         // Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
@@ -68,7 +67,7 @@
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);        
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
         if (anySecondaryKeyIsNullable) {
             spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
             spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
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 5a4d6c9..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
@@ -12,9 +12,9 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -26,11 +26,11 @@
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -66,7 +66,7 @@
     protected int numSecondaryKeys;
     protected AqlCompiledMetadataDeclarations metadata;
     protected String datasetName;
-    protected AqlCompiledDatasetDecl datasetDecl;
+    protected Dataset dataset;
     protected ARecordType itemType;
     protected ISerializerDeserializer payloadSerde;
     protected IFileSplitProvider primaryFileSplitProvider;
@@ -80,7 +80,7 @@
     protected RecordDescriptor primaryRecDesc;
     protected IBinaryComparatorFactory[] secondaryComparatorFactories;
     protected RecordDescriptor secondaryRecDesc;
-    protected ICopyEvaluatorFactory[] evalFactories;
+    protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
 
     // Prevent public construction. Should be created via createIndexCreator().
     protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf) {
@@ -100,7 +100,8 @@
                 indexCreator = new SecondaryRTreeCreator(physOptConf);
                 break;
             }
-            case KEYWORD: {
+            case WORD_INVIX:
+            case NGRAM_INVIX: {
                 indexCreator = new SecondaryInvertedIndexCreator(physOptConf);
                 break;
             }
@@ -121,16 +122,16 @@
         this.metadata = metadata;
         datasetName = createIndexStmt.getDatasetName();
         secondaryIndexName = createIndexStmt.getIndexName();
-        datasetDecl = metadata.findDataset(datasetName);
-        if (datasetDecl == null) {
+        dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Unknown dataset " + datasetName);
         }
-        if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
         }
-        itemType = (ARecordType) metadata.findType(datasetDecl.getItemTypeName());
+        itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         numSecondaryKeys = createIndexStmt.getKeyFields().size();
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
@@ -142,51 +143,49 @@
         secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
         // Must be called in this order.
         setPrimaryRecDescAndComparators();
-        setSecondaryRecDescAndComparators(createIndexStmt.getKeyFields());
+        setSecondaryRecDescAndComparators(createIndexStmt);
     }
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numPrimaryKeys = partitioningKeys.size();
         ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
         primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(datasetDecl);
-        int i = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
-            primaryRecFields[i] = keySerde;
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
+            primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
             primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
                     keyType, true);
             primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            ++i;
         }
         primaryRecFields[numPrimaryKeys] = payloadSerde;
         primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
         primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
     }
 
-    protected void setSecondaryRecDescAndComparators(List<String> secondaryKeyFields) throws AlgebricksException,
-            AsterixException {
-        evalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
+    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt)
+            throws AlgebricksException, AsterixException {
+        List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
+        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
         secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
         ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadata.getFormat().getTypeTraitProvider();
+        IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadata.getFormat()
+                .getBinaryComparatorFactoryProvider();
         for (int i = 0; i < numSecondaryKeys; i++) {
-            evalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType, secondaryKeyFields.get(i),
-                    numPrimaryKeys);
-            Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                    secondaryKeyFields.get(i), itemType);
+            secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
+                    secondaryKeyFields.get(i), numPrimaryKeys);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
             IAType keyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
             secondaryRecFields[i] = keySerde;
-            secondaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                    keyType, true);
-            secondaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+            secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
         }
         // Add serializers and comparators for primary index fields.
         for (int i = 0; i < numPrimaryKeys; i++) {
@@ -228,7 +227,8 @@
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                highKeyFields, true, true, new BTreeDataflowHelperFactory(), false,
+                NoOpOperationCallbackProvider.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -248,10 +248,10 @@
         for (int i = 0; i < numPrimaryKeys; i++) {
             projectionList[projCount++] = i;
         }
-        IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[evalFactories.length];
-        for (int i = 0; i < evalFactories.length; ++i) {
+        IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
+        for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
             sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
-                    evalFactories[i]);
+                    secondaryFieldAccessEvalFactories[i]);
         }
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
@@ -315,7 +315,7 @@
         }
         StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(
                 new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(selectCond),
-                null, AqlBinaryBooleanInspectorImpl.INSTANCE);
+                null, AqlBinaryBooleanInspectorImpl.FACTORY);
         AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index df2ef13..d5e8222 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -1,269 +1,222 @@
 package edu.uci.ics.asterix.file;
 
+import java.util.List;
+
+import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
+import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
+import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexCreateOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class SecondaryInvertedIndexCreator extends SecondaryIndexCreator {
-    
+
+    private IAType secondaryKeyType;
+    private ITypeTraits[] invListsTypeTraits;
+    private IBinaryComparatorFactory[] tokenComparatorFactories;
+    private ITypeTraits[] tokenTypeTraits;
+    private IBinaryTokenizerFactory tokenizerFactory;
+    private Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders;
+    // For tokenization, sorting and loading. Represents <token, primary keys>.
+    private int numTokenKeyPairFields;
+    private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
+    private RecordDescriptor tokenKeyPairRecDesc;
+
     protected SecondaryInvertedIndexCreator(PhysicalOptimizationConfig physOptConf) {
         super(physOptConf);
     }
 
     @Override
-    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-    
-    @Override
-    // TODO: This code has been completely rewritten in the asterix-fuzzy branch. No tests currently rely
-    // on this code, so I didn't do any cleanup here.
-    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        /*
-        JobSpecification spec = new JobSpecification();
-
-        String primaryIndexName = createIndexStmt.getDatasetName();
-        String secondaryIndexName = createIndexStmt.getIndexName();
-
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(primaryIndexName);
-        if (compiledDatasetDecl == null) {
-            throw new AsterixException("Could not find dataset " + primaryIndexName);
+    @SuppressWarnings("rawtypes")
+    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt)
+            throws AlgebricksException, AsterixException {
+        // Sanity checks.
+        if (numPrimaryKeys > 1) {
+            throw new AsterixException("Cannot create inverted index on dataset with composite primary key.");
         }
-
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AsterixException("Cannot index an external dataset (" + primaryIndexName + ").");
+        if (numSecondaryKeys > 1) {
+            throw new AsterixException("Cannot create composite inverted index on multiple fields.");
         }
-        ARecordType itemType = (ARecordType) metadata.findType(compiledDatasetDecl.getItemTypeName());
-        ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
-        ISerializerDeserializer payloadSerde = serdeProvider.getSerializerDeserializer(itemType);
-
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(compiledDatasetDecl).size();
-
-        // sanity
-        if (numPrimaryKeys > 1)
-            throw new AsterixException("Cannot create inverted keyword index on dataset with composite primary key.");
-
-        // sanity
-        IAType fieldsToTokenizeType = AqlCompiledIndexDecl
-                .keyFieldType(createIndexStmt.getKeyFields().get(0), itemType);
-        for (String fieldName : createIndexStmt.getKeyFields()) {
-            IAType nextFieldToTokenizeType = AqlCompiledIndexDecl.keyFieldType(fieldName, itemType);
-            if (nextFieldToTokenizeType.getTypeTag() != fieldsToTokenizeType.getTypeTag()) {
-                throw new AsterixException(
-                        "Cannot create inverted keyword index. Fields to tokenize must be of the same type.");
-            }
-        }
-
-        // ---------- START GENERAL BTREE STUFF
-
-        IIndexRegistryProvider<IIndex> treeRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
-        // ---------- END GENERAL BTREE STUFF
-
-        // ---------- START KEY PROVIDER OP
-
-        // TODO: should actually be empty tuple source
-        // build tuple containing low and high search keys
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(1); // just one dummy field
-        DataOutput dos = tb.getDataOutput();
-
-        try {
-            tb.reset();
-            IntegerSerializerDeserializer.INSTANCE.serialize(0, dos); // dummy
-            // field
-            tb.addFieldEndOffset();
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
-
-        ISerializerDeserializer[] keyRecDescSers = { IntegerSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> keyProviderSplitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(primaryIndexName, primaryIndexName);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, keyProviderOp,
-                keyProviderSplitsAndConstraint.second);
-
-        // ---------- END KEY PROVIDER OP
-
-        // ---------- START PRIMARY INDEX SCAN
-
-        ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
-        IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
-        ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
-        int i = 0;
-        for (Triple<IEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl)) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
-            primaryRecFields[i] = keySerde;
-			primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
-					.getBinaryComparatorFactory(keyType, true);
-            primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            ++i;
-        }
-        primaryRecFields[numPrimaryKeys] = payloadSerde;
-        primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
-
-        int[] lowKeyFields = null; // -infinity
-        int[] highKeyFields = null; // +infinity
-        RecordDescriptor primaryRecDesc = new RecordDescriptor(primaryRecFields);
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(primaryIndexName, primaryIndexName);
-
-        BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, treeRegistryProvider, primarySplitsAndConstraint.first, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
-                primarySplitsAndConstraint.second);
-
-        // ---------- END PRIMARY INDEX SCAN
-
-        // ---------- START ASSIGN OP
-
+        // Prepare record descriptor used in the assign op, and the optional select op.
         List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
-        int numSecondaryKeys = secondaryKeyFields.size();
+        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
-        IEvaluatorFactory[] evalFactories = new IEvaluatorFactory[numSecondaryKeys];
-        for (i = 0; i < numSecondaryKeys; i++) {
-            evalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
+        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+        ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
+        ITypeTraitProvider typeTraitProvider = metadata.getFormat().getTypeTraitProvider();
+        for (int i = 0; i < numSecondaryKeys; i++) {
+            secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
                     secondaryKeyFields.get(i), numPrimaryKeys);
-            IAType keyType = AqlCompiledIndexDecl.keyFieldType(secondaryKeyFields.get(i), itemType);
-            ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
+            secondaryKeyType = keyTypePair.first;
+            anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
+            ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
             secondaryRecFields[i] = keySerde;
+            secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
-        // fill in serializers and comparators for primary index fields
-        for (i = 0; i < numPrimaryKeys; i++) {
-            secondaryRecFields[numSecondaryKeys + i] = primaryRecFields[i];
+        secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        // Comparators and type traits for tokens.
+        tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
+        tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
+        tokenComparatorFactories[0] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
+        tokenTypeTraits[0] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+        // Set tokenizer factory.
+        // TODO: We might want to expose the hashing option at the AQL level, 
+        // and add the choice to the index metadata.
+        tokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(),
+                createIndexStmt.getIndexType(), createIndexStmt.getGramLength());
+        // Type traits for inverted-list elements. Inverted lists contain primary keys.
+        invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            invListsTypeTraits[i] = primaryRecDesc.getTypeTraits()[i];
         }
-        RecordDescriptor secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
-
-        int[] outColumns = new int[numSecondaryKeys];
-        int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
-        for (i = 0; i < numSecondaryKeys; i++) {
-            outColumns[i] = numPrimaryKeys + i + 1;
-        }
-        int projCount = 0;
-        for (i = 0; i < numSecondaryKeys; i++) {
-            projectionList[projCount++] = numPrimaryKeys + i + 1;
-        }
-        for (i = 0; i < numPrimaryKeys; i++) {
-            projectionList[projCount++] = i;
-        }
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> assignSplitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(primaryIndexName, primaryIndexName);
-
-        AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, evalFactories, projectionList);
-        AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
-                new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
-                assignSplitsAndConstraint.second);
-
-        // ---------- END ASSIGN OP
-
-        // ---------- START TOKENIZER OP
-
-        int numTokenKeyPairFields = numPrimaryKeys + 1;
-
+        // Get file split providers for the BTree and inverted-list files.
+        fileSplitProviders = metadata.getInvertedIndexFileSplitProviders(secondaryFileSplitProvider);
+        // For tokenization, sorting and loading.
+        // One token + primary keys.
+        numTokenKeyPairFields = 1 + numPrimaryKeys;
         ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
-        tokenKeyPairFields[0] = serdeProvider.getSerializerDeserializer(fieldsToTokenizeType);
-        for (i = 0; i < numPrimaryKeys; i++)
-            tokenKeyPairFields[i + 1] = secondaryRecFields[numSecondaryKeys + i];
-        RecordDescriptor tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields);
-
-        int[] fieldsToTokenize = new int[numSecondaryKeys];
-        for (i = 0; i < numSecondaryKeys; i++)
-            fieldsToTokenize[i] = i;
-
-        int[] primaryKeyFields = new int[numPrimaryKeys];
-        for (i = 0; i < numPrimaryKeys; i++)
-            primaryKeyFields[i] = numSecondaryKeys + i;
-
-        IBinaryTokenizerFactory tokenizerFactory = AqlBinaryTokenizerFactoryProvider.INSTANCE
-                .getTokenizerFactory(fieldsToTokenizeType);
-        BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenKeyPairRecDesc, tokenizerFactory, fieldsToTokenize, primaryKeyFields);
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(primaryIndexName, secondaryIndexName);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
-                secondarySplitsAndConstraint.second);
-
-        // ---------- END TOKENIZER OP
-
-        // ---------- START EXTERNAL SORT OP
-
-        IBinaryComparatorFactory[] tokenKeyPairComparatorFactories = new IBinaryComparatorFactory[numTokenKeyPairFields];
-		tokenKeyPairComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE
-				.getBinaryComparatorFactory(fieldsToTokenizeType, true);
-        for (i = 0; i < numPrimaryKeys; i++) {
+        ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
+        tokenKeyPairComparatorFactories = new IBinaryComparatorFactory[numTokenKeyPairFields];
+        tokenKeyPairFields[0] = serdeProvider.getSerializerDeserializer(secondaryKeyType);
+        tokenKeyPairTypeTraits[0] = tokenTypeTraits[0];
+        tokenKeyPairComparatorFactories[0] = InvertedIndexAccessMethod
+                .getTokenBinaryComparatorFactory(secondaryKeyType);
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            tokenKeyPairFields[i + 1] = primaryRecDesc.getFields()[i];
+            tokenKeyPairTypeTraits[i + 1] = primaryRecDesc.getTypeTraits()[i];
             tokenKeyPairComparatorFactories[i + 1] = primaryComparatorFactories[i];
         }
+        tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields, tokenKeyPairTypeTraits);
+    }
 
-        // <token, primarykey a, primarykey b, etc.>
-        int[] sortFields = new int[numTokenKeyPairFields]; 
-        for (i = 0; i < numTokenKeyPairFields; i++) {
-            sortFields[i] = i;
-        }
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sorterSplitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(primaryIndexName, primaryIndexName);
-
-        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                physOptConf.getMaxFramesExternalSort(), sortFields, tokenKeyPairComparatorFactories,
-                secondaryRecDesc);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp,
-                sorterSplitsAndConstraint.second);
-
-        // ---------- END EXTERNAL SORT OP
-
-        // ---------- START SECONDARY INDEX BULK LOAD
-
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numTokenKeyPairFields];
-        secondaryTypeTraits[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(fieldsToTokenizeType);
-        for (i = 0; i < numPrimaryKeys; i++)
-            secondaryTypeTraits[i + 1] = primaryTypeTraits[i];
-
-        int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
-        for (i = 0; i < numTokenKeyPairFields; i++)
-            fieldPermutation[i] = i;
-
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(
-                spec, storageManager, treeRegistryProvider,
-                secondarySplitsAndConstraint.first, secondaryTypeTraits,
-                tokenKeyPairComparatorFactories, fieldPermutation, 0.7f,
-                new BTreeDataflowHelperFactory(),
+    @Override
+    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
+        JobSpecification spec = new JobSpecification();
+        InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
+                AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first, fileSplitProviders.second,
+                AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
+                primaryComparatorFactories, tokenizerFactory, new BTreeDataflowHelperFactory(),
                 NoOpOperationCallbackProvider.INSTANCE);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryBulkLoadOp,
-                secondarySplitsAndConstraint.second);
-
-        // ---------- END SECONDARY INDEX BULK LOAD
-
-        // ---------- START CONNECT THE OPERATORS
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primarySearchOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), primarySearchOp, 0, asterixAssignOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, tokenizerOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), tokenizerOp, 0, sortOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-
-        spec.addRoot(secondaryBulkLoadOp);
-
-        // ---------- END CONNECT THE OPERATORS
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
+                secondaryPartitionConstraint);
+        spec.addRoot(invIndexCreateOp);
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         return spec;
-        */
-        return null;
+    }
+
+    @Override
+    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
+        JobSpecification spec = new JobSpecification();
+
+        // Create dummy key provider for feeding the primary index scan. 
+        AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+
+        // Create primary index scan op.
+        BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+
+        // Assign op.
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
+
+        // If any of the secondary fields are nullable, then add a select op that filters nulls.
+        AlgebricksMetaOperatorDescriptor selectOp = null;
+        if (anySecondaryKeyIsNullable) {
+            selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
+        }
+
+        // Create a tokenizer op.
+        AbstractOperatorDescriptor tokenizerOp = createTokenizerOp(spec);
+
+        // Sort by token + primary keys.
+        ExternalSortOperatorDescriptor sortOp = createSortOp(spec, tokenKeyPairComparatorFactories, tokenKeyPairRecDesc);
+
+        // Create secondary inverted index bulk load op.
+        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
+
+        // Connect the operators.
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+        if (anySecondaryKeyIsNullable) {
+            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+            spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, tokenizerOp, 0);
+        } else {
+            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, tokenizerOp, 0);
+        }
+        spec.connect(new OneToOneConnectorDescriptor(spec), tokenizerOp, 0, sortOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, invIndexBulkLoadOp, 0);
+        spec.addRoot(invIndexBulkLoadOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
+
+    private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
+        int[] fieldsToTokenize = new int[numSecondaryKeys];
+        for (int i = 0; i < numSecondaryKeys; i++) {
+            fieldsToTokenize[i] = i;
+        }
+        int[] primaryKeyFields = new int[numPrimaryKeys];
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            primaryKeyFields[i] = numSecondaryKeys + i;
+        }
+        BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenKeyPairRecDesc, tokenizerFactory, fieldsToTokenize, primaryKeyFields);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
+                primaryPartitionConstraint);
+        return tokenizerOp;
+    }
+
+    @Override
+    protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
+            IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
+        // Sort on token and primary keys.
+        int[] sortFields = new int[numTokenKeyPairFields];
+        for (int i = 0; i < numTokenKeyPairFields; i++) {
+            sortFields[i] = i;
+        }
+        ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
+                physOptConf.getMaxFramesExternalSort(), sortFields, tokenKeyPairComparatorFactories, secondaryRecDesc);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
+        return sortOp;
+    }
+
+    private InvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+        int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
+        for (int i = 0; i < numTokenKeyPairFields; i++) {
+            fieldPermutation[i] = i;
+        }
+        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
+                fieldPermutation, AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first,
+                fileSplitProviders.second, AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
+                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
+                secondaryPartitionConstraint);
+        return invIndexBulkLoadOp;
     }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 6980b14..3127573 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -9,9 +9,10 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -58,8 +59,9 @@
     }
 
     @Override
-    protected void setSecondaryRecDescAndComparators(List<String> secondaryKeyFields) throws AlgebricksException,
-            AsterixException {
+    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt)
+            throws AlgebricksException, AsterixException {
+        List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
         int numSecondaryKeys = secondaryKeyFields.size();
         if (numSecondaryKeys != 1) {
             throw new AsterixException(
@@ -67,8 +69,7 @@
                             + numSecondaryKeys
                             + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
         }
-        Pair<IAType, Boolean> spatialTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                secondaryKeyFields.get(0), itemType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
         IAType spatialType = spatialTypePair.first;
         anySecondaryKeyIsNullable = spatialTypePair.second;
         if (spatialType == null) {
@@ -76,8 +77,8 @@
         }
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         numNestedSecondaryKeyFields = numDimensions * 2;
-        evalFactories = metadata.getFormat().createMBRFactory(itemType, secondaryKeyFields.get(0), numPrimaryKeys,
-                numDimensions);
+        secondaryFieldAccessEvalFactories = metadata.getFormat().createMBRFactory(itemType, secondaryKeyFields.get(0),
+                numPrimaryKeys, numDimensions);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
index f755752..c295f55 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
@@ -146,7 +146,7 @@
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
                 storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
                 secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
         String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
         for (int p = 0; p < nodeGroup.size(); p++) {
             secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
@@ -218,10 +218,9 @@
 
         TestKeywordIndexJob tij = new TestKeywordIndexJob();
         JobSpecification jobSpec = tij.createJobSpec();
-        JobId jobId = hcc.createJob("asterix", jobSpec);
 
         long start = System.currentTimeMillis();
-        hcc.start(jobId);
+        JobId jobId = hcc.startJob("asterix", jobSpec);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
index 17d8c9f..5d2cdc5 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
@@ -146,7 +146,7 @@
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
                 storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
                 secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
         String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
         for (int p = 0; p < nodeGroup.size(); p++) {
             secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
@@ -218,10 +218,9 @@
 
         TestSecondaryIndexJob tij = new TestSecondaryIndexJob();
         JobSpecification jobSpec = tij.createJobSpec();
-        JobId jobId = hcc.createJob("asterix", jobSpec);
 
         long start = System.currentTimeMillis();
-        hcc.start(jobId);
+        JobId jobId = hcc.startJob("asterix", jobSpec);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
index 7e0ff86..d60971b 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
@@ -22,8 +22,7 @@
 
 import edu.uci.ics.asterix.api.common.AsterixHyracksIntegrationUtil;
 import edu.uci.ics.asterix.api.java.AsterixJavaClient;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.config.GlobalConfig;import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.test.base.AsterixTestHelper;
 import edu.uci.ics.asterix.test.common.TestHelper;
 
diff --git a/asterix-app/src/test/resources/optimizerts/ignore.txt b/asterix-app/src/test/resources/optimizerts/ignore.txt
index 49767eb..5e09322 100644
--- a/asterix-app/src/test/resources/optimizerts/ignore.txt
+++ b/asterix-app/src/test/resources/optimizerts/ignore.txt
@@ -1,2 +1,2 @@
 distinct_aggregate.aql
-cell-aggregation-with-filtering.aql
+cell-aggregation-with-filtering.aql
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-complex.aql b/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-complex.aql
new file mode 100644
index 0000000..313e9a4
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-complex.aql
@@ -0,0 +1,31 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/consolidate-complex-selects.aql";
+
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Transactions for Cooperative Environments")
+let $jacca := similarity-jaccard-check($paper_tokens, $query_tokens, 0.8f)
+let $jaccb := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
+let $jaccc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.3f)
+where $jacca[0] and $jaccb[0] and $paper.authors = "testauth" and $jaccc
+return {"Paper": $paper_tokens, "Query": $query_tokens }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-simple.aql b/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-simple.aql
new file mode 100644
index 0000000..8a237d2
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/consolidate-selects-simple.aql
@@ -0,0 +1,28 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type CustomerType as closed {
+  c_custkey: int32, 
+  c_name: string, 
+  c_address: string, 
+  c_nationkey: int32, 
+  c_phone: string, 
+  c_acctbal: double, 
+  c_mktsegment: string,
+  c_comment: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset Customers(CustomerType) 
+  partitioned by key c_custkey on group1;
+
+write output to nc1:"rttest/consolidate-selects-simple.aql";
+
+for $c in dataset('Customers')
+where $c.c_name = "testname" 
+  and $c.c_address = "testaddr"
+  and $c.c_nationkey = 1
+  and $c.c_phone = "123456789"
+return $c
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
new file mode 100644
index 0000000..66ef43b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+
+// Cannot optimize this query because the string constant is shorter than the gram length.
+for $o in dataset('DBLP')
+where contains($o.title, "Mu")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
new file mode 100644
index 0000000..0aa8056
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check-panic.aql
new file mode 100644
index 0000000..21e41f8
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check-panic.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm";
+
+// This query cannot be optimized with an index, based on the high edit distance.
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
new file mode 100644
index 0000000..ff3728d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
+
+for $o in dataset('DBLP')
+where edit-distance-check($o.authors, "Amihay Motro", 1)[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-panic.aql
new file mode 100644
index 0000000..c993c57
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-panic.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-edit-distance-panic.adm";
+
+// This query cannot be optimized with an index, based on the high edit distance.
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Amihay Motro") <= 5
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
new file mode 100644
index 0000000..a143cde
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
+
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Amihay Motro") <= 1
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
new file mode 100644
index 0000000..87de1fb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '1';
+
+for $o in dataset('DBLP')
+where $o.authors ~= "Amihay Motro"
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
new file mode 100644
index 0000000..3db4c92
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.8f';
+
+for $o in dataset('DBLP')
+where gram-tokens($o.title, 3, false) ~= gram-tokens("Transactions for Cooperative Environments", 3, false)
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
new file mode 100644
index 0000000..c39d531
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
+
+for $o in dataset('DBLP')
+where similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
new file mode 100644
index 0000000..3855ade
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
+
+for $o in dataset('DBLP')
+where similarity-jaccard(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false)) >= 0.5f
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
new file mode 100644
index 0000000..6a6700a
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
@@ -0,0 +1,40 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+
+// Index should not be applied because all list elements can be modified by 3 edit operations.
+for $c in dataset('Customers')
+where edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
new file mode 100644
index 0000000..f705a4a
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+
+for $c in dataset('Customers')
+where edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
new file mode 100644
index 0000000..1fc7906
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
@@ -0,0 +1,40 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+
+// Index should not be applied because all list elements can be modified by 3 edit operations.
+for $c in dataset('Customers')
+where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 3
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
new file mode 100644
index 0000000..70268f5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+
+for $c in dataset('Customers')
+where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 1
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
new file mode 100644
index 0000000..90ce266
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
@@ -0,0 +1,42 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '1';
+
+for $c in dataset('Customers')
+where $c.interests ~= ["computers", "wine", "walking"]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
new file mode 100644
index 0000000..09da02e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
@@ -0,0 +1,41 @@
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.8f';
+
+for $c in dataset('Customers')
+where $c.interests ~= ["databases", "computers", "wine"]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
new file mode 100644
index 0000000..18721c5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
@@ -0,0 +1,38 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
+
+for $c in dataset('Customers')
+where similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)[0]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
new file mode 100644
index 0000000..fa2291c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
@@ -0,0 +1,38 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
+
+for $c in dataset('Customers')
+where similarity-jaccard($c.interests, ["databases", "computers", "wine"]) >= 0.7f
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
new file mode 100644
index 0000000..89c2dac
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
@@ -0,0 +1,41 @@
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.8f';
+
+for $c in dataset('Customers')
+where $c.interests ~= ["databases", "computers", "wine"]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
new file mode 100644
index 0000000..8089d6d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
@@ -0,0 +1,38 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+
+for $c in dataset('Customers')
+where similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)[0]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
new file mode 100644
index 0000000..547f7e2
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
@@ -0,0 +1,38 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+
+for $c in dataset('Customers')
+where similarity-jaccard($c.interests, ["databases", "computers", "wine"]) >= 0.7f
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
new file mode 100644
index 0000000..0b5367b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_word-contains.adm";
+
+// Contains cannot be answered with a word inverted index.
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
new file mode 100644
index 0000000..39b247d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $o in dataset('DBLP')
+where word-tokens($o.title) ~= word-tokens("Transactions for Cooperative Environments")
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
new file mode 100644
index 0000000..f1d3d4b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
+
+for $o in dataset('DBLP')
+where similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
new file mode 100644
index 0000000..16fb8d3
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-basic_word-jaccard.adm";
+
+for $o in dataset('DBLP')
+where similarity-jaccard(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments")) >= 0.5f
+return $o
+
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
new file mode 100644
index 0000000..8c9500f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
@@ -0,0 +1,36 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
+ *                  two edit-distance-check function of which only the first can be optimized with an index. 
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+
+// Only the first edit-distance-check can be optimized with an index.
+for $o in dataset('DBLP')
+let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
+let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $eda[0] and $edb[0] 
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
new file mode 100644
index 0000000..55bd024
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
@@ -0,0 +1,36 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
+ *                  two edit-distance-check function of which only the second can be optimized with an index.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+
+// Only the second edit-distance-check can be optimized with an index.
+for $o in dataset('DBLP')
+let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
+let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
+where $edb[0] and $eda[0] 
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic.aql
new file mode 100644
index 0000000..458425f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm";
+
+// This query cannot be optimized with an index, based on the high edit distance.
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
new file mode 100644
index 0000000..f5c9a18
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
new file mode 100644
index 0000000..5d3758e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
+where $jacc[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
new file mode 100644
index 0000000..30d97b8
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
@@ -0,0 +1,36 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
+
+// This test is complex because we have three assigns to drill into.
+for $paper in dataset('DBLP')
+let $paper_tokens := gram-tokens($paper.title, 3, false)
+let $query_tokens := gram-tokens("Transactions for Cooperative Environments", 3, false)
+let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
+where $jacc[0]
+return {"Paper": $paper_tokens, "Query": $query_tokens }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
new file mode 100644
index 0000000..51e66a1
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
@@ -0,0 +1,41 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
new file mode 100644
index 0000000..2d6c4cd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
@@ -0,0 +1,41 @@
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
new file mode 100644
index 0000000..ff9d8e6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
@@ -0,0 +1,40 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
new file mode 100644
index 0000000..57d9244
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
@@ -0,0 +1,40 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
new file mode 100644
index 0000000..f79e10f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
new file mode 100644
index 0000000..465cb51
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
@@ -0,0 +1,36 @@
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
+
+// This test is complex because we have three assigns to drill into.
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Transactions for Cooperative Environments")
+let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.8f)
+where $jacc[0]
+return {"Paper": $paper_tokens, "Query": $query_tokens }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_01.aql
new file mode 100644
index 0000000..d802dc5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_01.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
new file mode 100644
index 0000000..01003f9
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
new file mode 100644
index 0000000..890dfa9
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_01.aql
new file mode 100644
index 0000000..76622d9
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_01.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
new file mode 100644
index 0000000..4298db1
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
new file mode 100644
index 0000000..72ca9d0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
new file mode 100644
index 0000000..14a4420
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors ~= $b.authors and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_02.aql
new file mode 100644
index 0000000..2ad1f2b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_02.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where $a.authors ~= $b.authors and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
new file mode 100644
index 0000000..640b6e1
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
@@ -0,0 +1,34 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where $a.authors ~= $b.authors and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
new file mode 100644
index 0000000..55af0e4
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
new file mode 100644
index 0000000..c00726e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
new file mode 100644
index 0000000..9d2ce15
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
@@ -0,0 +1,35 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
new file mode 100644
index 0000000..cb5f92c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
new file mode 100644
index 0000000..8f487ed
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
new file mode 100644
index 0000000..9c57be6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
new file mode 100644
index 0000000..14b52fb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
new file mode 100644
index 0000000..4c51967
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on CSX(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
new file mode 100644
index 0000000..cd84d30
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_01.aql
new file mode 100644
index 0000000..b4f8b56
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance-check_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
new file mode 100644
index 0000000..db47ed6b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
new file mode 100644
index 0000000..d97c003
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_01.aql
new file mode 100644
index 0000000..4047e55
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
new file mode 100644
index 0000000..5745565
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
new file mode 100644
index 0000000..fc3fc4a
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_01.aql
new file mode 100644
index 0000000..bf2b8bc
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_01.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
new file mode 100644
index 0000000..5314b95
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
new file mode 100644
index 0000000..15a9ab7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
@@ -0,0 +1,42 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '3';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_01.aql
new file mode 100644
index 0000000..f6f2f84
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_01.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
new file mode 100644
index 0000000..1951e6f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
new file mode 100644
index 0000000..d791b85
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
@@ -0,0 +1,42 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_01.aql
new file mode 100644
index 0000000..5f6f59b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard-check_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
new file mode 100644
index 0000000..0754282
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
new file mode 100644
index 0000000..4dbc4d5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_01.aql
new file mode 100644
index 0000000..ddf386e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
new file mode 100644
index 0000000..50c3db6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
new file mode 100644
index 0000000..50729ba
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_01.aql
new file mode 100644
index 0000000..1fa479d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_01.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_01.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
new file mode 100644
index 0000000..e5b532f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
new file mode 100644
index 0000000..a881c89
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
@@ -0,0 +1,42 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.7f';
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where $a.interests ~= $b.interests and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_01.aql
new file mode 100644
index 0000000..5d95894
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard-check_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
new file mode 100644
index 0000000..561e15f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
new file mode 100644
index 0000000..87d78e5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_01.aql
new file mode 100644
index 0000000..864ede7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_01.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard_01.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
new file mode 100644
index 0000000..4d9f89e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
@@ -0,0 +1,45 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers2(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
new file mode 100644
index 0000000..5eae45b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
@@ -0,0 +1,39 @@
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers')
+where similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql
new file mode 100644
index 0000000..6ea53e9
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
new file mode 100644
index 0000000..254a825
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
@@ -0,0 +1,49 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on CSX(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
new file mode 100644
index 0000000..f553abe
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
@@ -0,0 +1,35 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql
new file mode 100644
index 0000000..63efb89
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
new file mode 100644
index 0000000..1298833
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on CSX(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
new file mode 100644
index 0000000..c2d6b78
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql
new file mode 100644
index 0000000..7c69928
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
+
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
new file mode 100644
index 0000000..1f5c082
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on CSX(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
+
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
new file mode 100644
index 0000000..94e59c0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('DBLP')
+where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
+      and $a.id < $b.id
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_01.aql
new file mode 100644
index 0000000..a456568
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_01.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a "<=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_01.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where $ed <= 2
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_02.aql
new file mode 100644
index 0000000..be946af
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_02.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause. 
+ *                  Replacement should happen because of a reverse "<=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_02.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where 2 >= $ed
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_03.aql
new file mode 100644
index 0000000..e709e8c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_03.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a "<" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_03.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where $ed < 3
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_04.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_04.aql
new file mode 100644
index 0000000..8bbd3d6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_04.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a reverse "<" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_04.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where 3 > $ed
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_05.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_05.aql
new file mode 100644
index 0000000..b929ef2
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_05.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a ">=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_05.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where $ed >= 2
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_06.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_06.aql
new file mode 100644
index 0000000..3b86bd0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_06.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                   Replacement should *not* happen because of a reverse ">=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_06.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where 2 <= $ed
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_07.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_07.aql
new file mode 100644
index 0000000..9960262
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_07.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a ">" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_07.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where $ed > 2
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_08.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_08.aql
new file mode 100644
index 0000000..ddb5018
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-let-to-edit-distance-check_08.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a reverse ">" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_08.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance($o.authors, "Michael J. Carey")
+where 2 < $ed
+return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_01.aql
new file mode 100644
index 0000000..4a21a3b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_01.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should happen because of a "<=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_01.adm";
+
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Michael J. Carey") <= 2
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_02.aql
new file mode 100644
index 0000000..6657bf4
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_02.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should happen because of a reverse "<=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_02.adm";
+
+for $o in dataset('DBLP')
+where 2 >= edit-distance($o.authors, "Michael J. Carey")
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_03.aql
new file mode 100644
index 0000000..f1796a1
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_03.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should happen because of a "<" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_03.adm";
+
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Michael J. Carey") < 3
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_04.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_04.aql
new file mode 100644
index 0000000..fb63621
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_04.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should happen because of a reverse "<" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_04.adm";
+
+for $o in dataset('DBLP')
+where 3 > edit-distance($o.authors, "Michael J. Carey")
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_05.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_05.aql
new file mode 100644
index 0000000..b421d54
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_05.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should *not* happen because of a ">=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_05.adm";
+
+// We cannot introduce edit-distance-check because the condition is >=
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Michael J. Carey") >= 2
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_06.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_06.aql
new file mode 100644
index 0000000..a9372ec
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_06.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should *not* happen because of a reverse ">=" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_06.adm";
+
+// We cannot introduce edit-distance-check because the condition is <=
+for $o in dataset('DBLP')
+where 2 <= edit-distance($o.authors, "Michael J. Carey")
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_07.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_07.aql
new file mode 100644
index 0000000..74959c7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_07.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should *not* happen because of a ">" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_07.adm";
+
+// We cannot introduce edit-distance-check because the condition is >
+for $o in dataset('DBLP')
+where edit-distance($o.authors, "Michael J. Carey") > 2
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_08.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_08.aql
new file mode 100644
index 0000000..4ac54ed
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/edit-distance-to-edit-distance-check_08.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the edit-distance function with a threshold 
+ *                  into edit-distance-check if possible.
+ *                  Replacement should *not* happen because of a reverse ">" condition on the edit distance.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_edit-distance-to-edit-distance-check_08.adm";
+
+// We cannot introduce edit-distance-check because the condition is <
+for $o in dataset('DBLP')
+where 2 < edit-distance($o.authors, "Michael J. Carey")
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-edit-distance-check.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-edit-distance-check.aql
new file mode 100644
index 0000000..f53435b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-edit-distance-check.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests that the FuzzyEqRule rewrites ~= using edit distance 
+ *                  into edit-distance-check.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_fuzzyeq-to-edit-distance-check.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '1';
+
+// Tests two rules: FuzzyEqRule and SimilarityCheckRule
+for $o in dataset('DBLP')
+where $o.authors ~= "Michael J. Carey"
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-jaccard-check.aql
new file mode 100644
index 0000000..f4da76f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/fuzzyeq-to-jaccard-check.aql
@@ -0,0 +1,33 @@
+/*
+ * Description    : Tests that the FuzzyEqRule rewrites ~= using Jaccard 
+ *                  into edit-distance-check.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_fuzzyeq-to-jaccard-check.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.8f';
+
+// Tests two rules: FuzzyEqRule and SimilarityCheckRule
+for $paper in dataset('DBLP')
+where word-tokens($paper.title) ~= word-tokens("Transactions for Cooperative Environments")
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_01.aql
new file mode 100644
index 0000000..c76f15b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_01.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a ">=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where $jacc >= 0.8f
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_02.aql
new file mode 100644
index 0000000..52296d5
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_02.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a reverse ">=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where 0.8f <= $jacc 
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_03.aql
new file mode 100644
index 0000000..8251c76
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_03.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a ">" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where $jacc > 0.8f
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_04.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_04.aql
new file mode 100644
index 0000000..2c53231
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_04.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should happen because of a reverse ">" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where 0.8f < $jacc 
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_05.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_05.aql
new file mode 100644
index 0000000..03a0321
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_05.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a "<=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where $jacc <= 0.8f
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_06.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_06.aql
new file mode 100644
index 0000000..b1d26da
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_06.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a reverse "<=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where 0.8f >= $jacc 
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_07.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_07.aql
new file mode 100644
index 0000000..e008feb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_07.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a "<" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where $jacc < 0.8f
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_08.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_08.aql
new file mode 100644
index 0000000..7e93d7f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-let-to-jaccard-check_08.aql
@@ -0,0 +1,32 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Tests that the optimizer drills through the let clause.
+ *                  Replacement should *not* happen because of a reverse "<" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+let $jacc := similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+where 0.8f > $jacc 
+return $jacc
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_01.aql
new file mode 100644
index 0000000..ecc0554
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_01.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should happen because of a ">=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_01.adm";
+
+for $paper in dataset('DBLP')
+where similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments")) >= 0.8f
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_02.aql
new file mode 100644
index 0000000..18bf1a3
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_02.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should happen because of a reverse ">=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_02.adm";
+
+for $paper in dataset('DBLP')
+where 0.8f <= similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_03.aql
new file mode 100644
index 0000000..f2f6e77
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_03.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should happen because of a ">" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_02.adm";
+
+for $paper in dataset('DBLP')
+where similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments")) > 0.8f
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_04.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_04.aql
new file mode 100644
index 0000000..8a945bc
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_04.aql
@@ -0,0 +1,30 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should happen because of a reverse ">" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_03.adm";
+
+for $paper in dataset('DBLP')
+where 0.8f < similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_05.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_05.aql
new file mode 100644
index 0000000..e9808ff
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_05.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should *not* happen because of a "<=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_05.adm";
+
+// We cannot introduce jaccard-check because the condition is <=
+for $paper in dataset('DBLP')
+where similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments")) <= 0.8f
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_06.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_06.aql
new file mode 100644
index 0000000..91953fc
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_06.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should *not* happen because of a reverse "<=" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_06.adm";
+
+// We cannot introduce jaccard-check because the condition is >=
+for $paper in dataset('DBLP')
+where 0.8f >= similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_07.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_07.aql
new file mode 100644
index 0000000..2f3b080
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_07.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should *not* happen because of a "<" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_07.adm";
+
+// We cannot introduce jaccard-check because the condition is <
+for $paper in dataset('DBLP')
+where similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments")) < 0.8f
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_08.aql b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_08.aql
new file mode 100644
index 0000000..fc61d1f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-to-jaccard-check_08.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Tests that the SimilarityCheckRule rewrites the similarity-jaccard function with a threshold 
+ *                  into similarity-jaccard-check-check if possible.
+ *                  Replacement should *not* happen because of a reverse "<" condition on the similarity.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_jaccard-to-jaccard-check_08.adm";
+
+// We cannot introduce jaccard-check because the condition is >
+for $paper in dataset('DBLP')
+where 0.8f > similarity-jaccard(word-tokens($paper.title), word-tokens("Transactions for Cooperative Environments"))
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_01.aql b/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_01.aql
new file mode 100644
index 0000000..85ee87a
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_01.aql
@@ -0,0 +1,15 @@
+/*
+ * Description    : Tests that non-datascan unnests are rewritten into joins.
+ * Success        : Yes
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/unnest-to-join_01.adm";
+
+for $x in [1,2,3,4,5,6]
+for $y in [4,5,6,7,8,9]
+where $x = $y
+return $y
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_02.aql b/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_02.aql
new file mode 100644
index 0000000..2402eb0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/unnest-to-join_02.aql
@@ -0,0 +1,12 @@
+/*
+ * Description    : Tests that non-datascan unnests in a subplan are rewritten into joins.
+ * Success        : Yes
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/unnest-to-join_02.adm";
+
+some $x in [1,2,3,4,5,6], $y in [4,5,6,7,8,9] satisfies $x = $y
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
new file mode 100644
index 0000000..f3e404f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
@@ -0,0 +1,12 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan b/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
index 3130442..3d1699e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
@@ -24,12 +24,12 @@
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- STABLE_SORT [$$46(ASC), $$6(ASC)]  |LOCAL|
+                                  -- STABLE_SORT [$$46(ASC), $$5(ASC)]  |LOCAL|
                                     -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$4][$$26]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$3][$$26]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- UNNEST  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
@@ -63,12 +63,12 @@
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
                                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                  -- STABLE_SORT [$$47(ASC), $$15(ASC)]  |LOCAL|
+                                  -- STABLE_SORT [$$47(ASC), $$14(ASC)]  |LOCAL|
                                     -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$13][$$35]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$12][$$35]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- UNNEST  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
index 3042fae..c930bc7 100644
--- a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
@@ -10,12 +10,12 @@
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-              -- STABLE_SORT [$$23(ASC), $$7(ASC)]  |LOCAL|
+              -- STABLE_SORT [$$23(ASC), $$6(ASC)]  |LOCAL|
                 -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$2][$$7]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$1][$$6]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- UNNEST  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
@@ -23,7 +23,7 @@
                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$7]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
                           -- RUNNING_AGGREGATE  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- SORT_MERGE_EXCHANGE [$$24(DESC) ]  |PARTITIONED|
@@ -35,13 +35,13 @@
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                             }
                                       -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$30(ASC)] HASH:[$$30]  |PARTITIONED|
-                                        -- PRE_CLUSTERED_GROUP_BY[$$6]  |LOCAL|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$5]  |LOCAL|
                                                 {
                                                   -- AGGREGATE  |LOCAL|
                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                 }
                                           -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                                            -- STABLE_SORT [$$6(ASC)]  |LOCAL|
+                                            -- STABLE_SORT [$$5(ASC)]  |LOCAL|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- UNNEST  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan b/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
index bac7a04..ef6d4e3 100644
--- a/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
@@ -9,14 +9,14 @@
                 -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
                         {
                           -- AGGREGATE  |LOCAL|
-                            -- IN_MEMORY_STABLE_SORT [$$5(ASC)]  |LOCAL|
+                            -- IN_MEMORY_STABLE_SORT [$$4(ASC)]  |LOCAL|
                               -- STREAM_SELECT  |LOCAL|
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                         }
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- IN_MEMORY_HASH_JOIN [$$3][$$8]  |PARTITIONED|
+                        -- IN_MEMORY_HASH_JOIN [$$2][$$7]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- UNNEST  |PARTITIONED|
@@ -29,8 +29,8 @@
                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                             -- RUNNING_AGGREGATE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- SORT_MERGE_EXCHANGE [$$28(ASC), $$8(ASC) ]  |PARTITIONED|
-                                  -- STABLE_SORT [$$28(ASC), $$8(ASC)]  |LOCAL|
+                                -- SORT_MERGE_EXCHANGE [$$28(ASC), $$7(ASC) ]  |PARTITIONED|
+                                  -- STABLE_SORT [$$28(ASC), $$7(ASC)]  |LOCAL|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EXTERNAL_GROUP_BY[$$36]  |PARTITIONED|
                                               {
@@ -38,7 +38,7 @@
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
                                         -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                                          -- EXTERNAL_GROUP_BY[$$7]  |LOCAL|
+                                          -- EXTERNAL_GROUP_BY[$$6]  |LOCAL|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
new file mode 100644
index 0000000..4efec2d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
new file mode 100644
index 0000000..356b847
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
new file mode 100644
index 0000000..f3b90fe
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$8(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
new file mode 100644
index 0000000..f3b90fe
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$8(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
new file mode 100644
index 0000000..77c98be
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$7(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
new file mode 100644
index 0000000..5f5331c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
new file mode 100644
index 0000000..5f5331c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
new file mode 100644
index 0000000..4c01aae
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
new file mode 100644
index 0000000..e0176c7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$11(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
new file mode 100644
index 0000000..4c01aae
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
new file mode 100644
index 0000000..e0176c7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$11(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
new file mode 100644
index 0000000..854f188
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$6(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
new file mode 100644
index 0000000..f3b90fe
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$8(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
new file mode 100644
index 0000000..f3b90fe
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$8(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
new file mode 100644
index 0000000..4efec2d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
new file mode 100644
index 0000000..fd84553
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
new file mode 100644
index 0000000..5f5331c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
new file mode 100644
index 0000000..5f5331c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -0,0 +1,16 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
new file mode 100644
index 0000000..0e38aba
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$15(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
new file mode 100644
index 0000000..0e38aba
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$15(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
new file mode 100644
index 0000000..816c9f0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$9(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
new file mode 100644
index 0000000..db6d7f0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$11(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
new file mode 100644
index 0000000..a2ce938
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                      -- STABLE_SORT [$$14(ASC)]  |LOCAL|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
new file mode 100644
index 0000000..1d1f5ca
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
new file mode 100644
index 0000000..543d70d
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -0,0 +1,18 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$12(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
new file mode 100644
index 0000000..19748ce
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
new file mode 100644
index 0000000..19748ce
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$10(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
new file mode 100644
index 0000000..db6d7f0
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -0,0 +1,17 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STABLE_SORT [$$11(ASC)]  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
new file mode 100644
index 0000000..a2ce938
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                      -- STABLE_SORT [$$14(ASC)]  |LOCAL|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
new file mode 100644
index 0000000..b9f157b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$22(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
new file mode 100644
index 0000000..29e7a4f
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -0,0 +1,44 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- UNION_ALL  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+              -- STREAM_PROJECT  |LOCAL|
+                -- STREAM_SELECT  |LOCAL|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- NESTED_LOOP  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- DATASOURCE_SCAN  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- STREAM_SELECT  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SPLIT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
new file mode 100644
index 0000000..8e5b300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
new file mode 100644
index 0000000..9a4269c
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
new file mode 100644
index 0000000..9765cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
new file mode 100644
index 0000000..64a8027
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -0,0 +1,19 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |LOCAL|
+          -- STREAM_SELECT  |LOCAL|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index 6c99d34..6e22ff1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -8,11 +8,11 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index 7aff2c2..2a501cf 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                    -- STABLE_SORT [$$22(ASC)]  |LOCAL|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index 6c99d34..c9d9f78 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                    -- STABLE_SORT [$$19(ASC)]  |LOCAL|
+                    -- STABLE_SORT [$$20(ASC)]  |LOCAL|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index 7aff2c2..85badf2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -8,11 +8,11 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                    -- STABLE_SORT [$$21(ASC)]  |LOCAL|
+                    -- STABLE_SORT [$$22(ASC)]  |LOCAL|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
index ce5cbbf..754f055 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
@@ -6,7 +6,7 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                -- STABLE_SORT [$$13(ASC)]  |LOCAL|
+                -- STABLE_SORT [$$14(ASC)]  |LOCAL|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
index ce5cbbf..754f055 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
@@ -6,7 +6,7 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                -- STABLE_SORT [$$13(ASC)]  |LOCAL|
+                -- STABLE_SORT [$$14(ASC)]  |LOCAL|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
index 2b4529d..87ee225 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
@@ -7,11 +7,11 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                  -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                  -- STABLE_SORT [$$22(ASC)]  |LOCAL|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- RTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
index 2b4529d..8bffa2b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-                  -- STABLE_SORT [$$18(ASC)]  |LOCAL|
+                  -- STABLE_SORT [$$22(ASC)]  |LOCAL|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index ede1063..03fd04e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -15,9 +15,8 @@
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan
new file mode 100644
index 0000000..170edbd
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan
@@ -0,0 +1,11 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan
new file mode 100644
index 0000000..3babdef
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan
new file mode 100644
index 0000000..467de52
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan
@@ -0,0 +1,9 @@
+-- SINK_WRITE  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_01.plan
new file mode 100644
index 0000000..d43cc94
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_01.plan
@@ -0,0 +1,10 @@
+-- SINK_WRITE  |UNPARTITIONED|
+  -- STREAM_PROJECT  |UNPARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+      -- HYBRID_HASH_JOIN [$$0][$$1]  |UNPARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+          -- UNNEST  |UNPARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+          -- UNNEST  |UNPARTITIONED|
+            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_02.plan
new file mode 100644
index 0000000..bfb15c7
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/unnest-to-join_02.plan
@@ -0,0 +1,21 @@
+-- SINK_WRITE  |UNPARTITIONED|
+  -- STREAM_PROJECT  |UNPARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+      -- PRE_CLUSTERED_GROUP_BY[$$8]  |UNPARTITIONED|
+              {
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
+              }
+        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+          -- STABLE_SORT [$$8(ASC)]  |LOCAL|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- STREAM_PROJECT  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$0][$$1]  |UNPARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan b/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
index 0a2d8a3..d249c29 100644
--- a/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
@@ -14,8 +14,8 @@
                 -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$4][$$22]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$3][$$22]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- UNNEST  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/ignore.txt b/asterix-app/src/test/resources/runtimets/ignore.txt
index a850d3c..1cf8519 100644
--- a/asterix-app/src/test/resources/runtimets/ignore.txt
+++ b/asterix-app/src/test/resources/runtimets/ignore.txt
@@ -1,8 +1,14 @@
 custord/join_q_04.aql
 scan/spatial_types_02.aql
 scan/temp_types_02.aql
-fuzzyjoin/dblp-csx-3_5.4.aql
 fuzzyjoin/dblp-splits-3_1.aql
+fuzzyjoin/dblp-csx-aqlplus_1.aql
+fuzzyjoin/dblp-csx-aqlplus_2.aql
+fuzzyjoin/dblp-csx-aqlplus_3.aql
+fuzzyjoin/events-users-aqlplus_1.aql
+fuzzyjoin/dblp-aqlplus_1.aql
+fuzzyjoin/dblp-csx-dblp-aqlplus_1.aql
+fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.aql
 subset-collection/04.aql
 quantifiers/everysat_01.aql
 custord/freq-clerk.aql
@@ -11,8 +17,6 @@
 custord/co.aql
 comparison/numeric-comparison_01.aql
 dapd/q3.aql
-fuzzyjoin/events-users-aqlplus_1.aql
-fuzzyjoin
 failure/q1_pricing_summary_report_failure.aql
 dml/load-from-hdfs.aql
 open-closed/open-closed-15
@@ -28,3 +32,8 @@
 open-closed/heterog-list02
 open-closed/heterog-list03
 open-closed/c2c
+quantifiers/somesat_03.aql
+quantifiers/somesat_04.aql
+quantifiers/somesat_05.aql
+quantifiers/everysat_02.aql
+quantifiers/everysat_03.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/add-null.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/add-null.aql
new file mode 100644
index 0000000..caa30d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/add-null.aql
@@ -0,0 +1,12 @@
+/*
+ * Description      :   Add anything plus null, the result should be null.
+ * Expected Result  :   Success
+ * Date             :   19th July 2012
+ */
+
+write output to nc1:"rttest/constructor_add-null.adm";
+
+let $x := 1
+let $y := 10
+let $z := 20
+return ($x+$y+$z+null)
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/drop-index.aql b/asterix-app/src/test/resources/runtimets/queries/dml/drop-index.aql
new file mode 100644
index 0000000..0c87734
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/drop-index.aql
@@ -0,0 +1,50 @@
+/*
+ * Description     : Drop secondary index.
+ * Expected Result : Success
+ * Date            : 12th July 2012
+ *
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Schema as closed {
+unique1:  int32,
+unique2:  int32,
+two:  int32,
+four:  int32,
+ten:  int32,
+twenty:  int32,
+onePercent: int32,
+tenPercent:  int32,
+twentyPercent:  int32,
+fiftyPercent:  int32,
+unique3:  int32,
+evenOnePercent: int32,
+oddOnePercent:  int32,
+stringu1:  string,
+stringu2:  string,
+string4:  string
+}
+
+create dataset t1(Schema) partitioned by key unique2;
+
+// Load data
+load dataset t1
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/wisc/onektup.adm"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+// create secondary indexes
+create index idx_t1_str1 on t1(stringu1);
+create index idx_t1_unique1 on t1(unique1);
+
+// drop secondary indexes
+drop index t1.idx_t1_str1;
+drop index t1.idx_t1_unique1;
+
+write output to nc1:"rttest/dml_drop-index.adm";
+
+for $a in dataset('t1')
+where $a.unique1 > 10 and $a.stringu1="DGAAAAXXXXXXXXXXXXXXXXXXX"
+return $a
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-aqlplus_1.aql
index 445b3f6..fa0bfac 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-aqlplus_1.aql
@@ -14,7 +14,9 @@
 
 create dataset DBLP(DBLPType) partitioned by key id;
 
-load dataset DBLP from nc1:'data/pub-small/dblp-small-id.txt' delimited by ':';
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
 write output to nc1:'rttest/fuzzyjoin_dblp-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_1.aql
index c5d4669..30445bf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_1.aql
@@ -20,11 +20,16 @@
   misc: string
 }
 
-create dataset DBLP(DBLPType) partitioned by key;
-create dataset CSX(CSXType) partitioned by key;
+create dataset DBLP(DBLPType) partitioned by key id;
+create dataset CSX(CSXType) partitioned by key id;
 
-load dataset DBLP from nc1:'data/pub-small/dblp-small-id.txt' delimited by ':';
-load dataset CSX from nc1:'data/pub-small/csx-small-id.txt' delimited by ':';
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
 write output to nc1:'rttest/fuzzyjoin_dblp-csx-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_2.aql
index da7d3cc..01ee281 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_2.aql
@@ -20,11 +20,16 @@
   misc: string
 }
 
-create dataset DBLP(DBLPType) partitioned by key;
-create dataset CSX(CSXType) partitioned by key;
+create dataset DBLP(DBLPType) partitioned by key id;
+create dataset CSX(CSXType) partitioned by key id;
 
-load dataset DBLP from nc1:'data/pub-small/dblp-small-id.txt' delimited by ':';
-load dataset CSX from nc1:'data/pub-small/csx-small-id.txt' delimited by ':';
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
 write output to nc1:'rttest/fuzzyjoin_dblp-csx-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_3.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_3.aql
index 0f0e993..4439af1 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_3.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_3.aql
@@ -23,8 +23,13 @@
 create dataset DBLP(DBLPType) partitioned by key id;
 create dataset CSX(CSXType) partitioned by key id;
 
-load dataset DBLP from nc1:'data/pub-small/dblp-small-id.txt' delimited by ':';
-load dataset CSX from nc1:'data/pub-small/csx-small-id.txt' delimited by ':';
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
 write output to nc1:'rttest/fuzzyjoin_dblp-csx-aqlplus_3.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-dblp-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-dblp-aqlplus_1.aql
index b6da9586..77e32a2 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-dblp-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-dblp-aqlplus_1.aql
@@ -23,8 +23,13 @@
 create dataset DBLP(DBLPType) partitioned by key id;
 create dataset CSX(CSXType) partitioned by key id;
 
-load dataset DBLP from nc1:'data/pub-small/dblp-small-id.txt' delimited by ':';
-load dataset CSX from nc1:'data/pub-small/csx-small-id.txt' delimited by ':';
+load dataset DBLP
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
 write output to nc1:'rttest/fuzzyjoin_dblp-csx-dblp-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-lookup_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-lookup_1.aql
index 61ca78e..274f0c2 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-lookup_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-lookup_1.aql
@@ -18,7 +18,6 @@
 using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
 (("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
 
-
 write output to nc1:'rttest/fuzzyjoin_dblp-lookup_1.adm';
 
 for $paper in dataset('DBLP')
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/events-users-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/events-users-aqlplus_1.aql
index b0c70ad..500454f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/events-users-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/events-users-aqlplus_1.aql
@@ -2,7 +2,6 @@
 create dataverse fuzzyjoin;
 use dataverse fuzzyjoin;
 
-
 create type AddressType as closed {
  street: string,
  city: string,
@@ -12,21 +11,22 @@
 
 create type UserType as open{
  name: string,
- interests: <string>,
+ interests: {{string}},
  address: AddressType,
- member_of: <
+ member_of: {{
   {
     sig_id: int32,
     chapter_name: string,
     member_since: date
   }
->
+}}
 }
 
+create dataset User(UserType) partitioned by key name;
 
-create dataset User(UserType)
- partitioned by key name;
-load dataset User from nc1:'data/events/tiny/user.adm';
+load dataset User
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/events/tiny/user.adm"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_events-users-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_1.aql
index b7e2d25e..7598987 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_1.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-int-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_2.aql
index 8910d67..5b3c828 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_2.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-int-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_3.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_3.aql
index f896430..c131c75 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_3.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-int-aqlplus_3.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-int-aqlplus_3.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.1.aql
index 584014a..6a6248f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.1.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-lot-aqlplus_1.1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.aql
index 2a9e544..75eb9c8 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_1.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-lot-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_2.aql
index c4592a2..39168c6 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_2.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-lot-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_3.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_3.aql
index 10d9c61..c9e4ece 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_3.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-lot-aqlplus_3.aql
@@ -6,12 +6,14 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-lot-aqlplus_3.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-3_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-3_1.aql
index d2aebca..fa63e8e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-3_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-3_1.aql
@@ -21,7 +21,6 @@
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-
 load dataset Users 
 using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
 (("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_1.aql
index 64fd639..5cea93e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_1.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-int-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_2.aql
index aa24266..6ec3fc6 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_2.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-int-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_3.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_3.aql
index 674ae07..baf6d12 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_3.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-aqlplus_3.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-int-aqlplus_3.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.aql
index 51cbc6d..ff94813 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-int-vis-user-lot-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_1.aql
index 439bf67..1cfaccf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_1.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_2.aql
index 81b0b60..b5eb560 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_2.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_3.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_3.aql
index 3c98faa..5b557ee 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_3.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_3.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-aqlplus_3.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_4.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_4.aql
index 5e12c6d..1353960 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_4.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_4.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-aqlplus_4.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_5.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_5.aql
index 8f14385..58cc81f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_5.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-aqlplus_5.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-aqlplus_5.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_1.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_1.aql
index facf238..01ad330 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_1.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_1.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-int-aqlplus_1.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_2.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_2.aql
index e38178e..35fde70 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_2.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/user-vis-lot-int-aqlplus_2.aql
@@ -8,21 +8,26 @@
   uid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create type VisitorType as open {
   vid: int32, 
   name: string,
   lottery_numbers: [int32],
-  interests: <string>
+  interests: {{string}}
 }
 
 create dataset Users(UserType) partitioned by key uid;
 create dataset Visitors(VisitorType) partitioned by key vid;
 
-load dataset Users from nc1:'data/users-visitors-small/users.json';
-load dataset Visitors from nc1:'data/users-visitors-small/visitors.json';
+load dataset Users 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/users.json"),("format"="adm"));
+
+load dataset Visitors 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/users-visitors-small/visitors.json"),("format"="adm"));
 
 write output to nc1:'rttest/fuzzyjoin_user-vis-lot-int-aqlplus_2.adm';
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-edit-distance.aql
new file mode 100644
index 0000000..dfd86e3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-edit-distance.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customers and Customers2, based on the edit-distance function of their names.
+ *                  Customers has a 3-gram index on name, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as open {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index ngram_index on Customers(name) type ngram(3);
+
+write output to nc1:"rttest/index-join_inverted-index-ngram-edit-distance.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where edit-distance($a.name, $b.name) <= 4 and $a.cid < $b.cid
+order by $a.cid, $b.cid
+return { "arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-jaccard.aql
new file mode 100644
index 0000000..6f69866
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ngram-jaccard.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/index-join_inverted-index-ngram-jaccard.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f 
+      and $a.id < $b.id
+order by $a.id, $b.id
+return { "arec": $a.title, "brec": $b.title }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-edit-distance.aql
new file mode 100644
index 0000000..601d1b8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-edit-distance.aql
@@ -0,0 +1,46 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customers and Customers2, based on the edit-distance function of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as open {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-join_inverted-index-olist-edit-distance.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where len($a.interests) > 2 and len($b.interests) > 2 and edit-distance($a.interests, $b.interests) <= 1 and $a.cid < $b.cid
+order by $a.cid, $b.cid
+return { "arec": $a, "brec": $b }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-jaccard.aql
new file mode 100644
index 0000000..91fcd80
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-olist-jaccard.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customers and Customers2, based on the Jaccard similarity of their interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-join_inverted-index-olist-jaccard.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.9f 
+      and $a.cid < $b.cid
+order by $a.cid, $b.cid
+return { "a": $a.interests, "b": $b.interests }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ulist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ulist-jaccard.aql
new file mode 100644
index 0000000..2b2d52c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-ulist-jaccard.aql
@@ -0,0 +1,47 @@
+/*
+ * Description    : Fuzzy joins two datasets, Customers and Customers2, based on the Jaccard similarity of their interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+  
+create dataset Customers2(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+load dataset Customers2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-join_inverted-index-ulist-jaccard.adm";
+
+for $a in dataset('Customers')
+for $b in dataset('Customers2')
+where similarity-jaccard($a.interests, $b.interests) >= 0.9f 
+      and $a.cid < $b.cid
+order by $a.cid, $b.cid
+return { "a": $a.interests, "b": $b.interests }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-word-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-word-jaccard.aql
new file mode 100644
index 0000000..228dfd2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-join/inverted-index-word-jaccard.aql
@@ -0,0 +1,48 @@
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) partitioned by key id;
+
+create dataset CSX(CSXType) partitioned by key id;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/index-join_inverted-index-word-jaccard.adm";
+
+for $a in dataset('DBLP')
+for $b in dataset('CSX')
+where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f 
+      and $a.id < $b.id
+order by $a.id, $b.id
+return { "arec": $a.title, "brec": $b.title }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable.aql
new file mode 100644
index 0000000..460a212
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable.aql
@@ -0,0 +1,33 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as open {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create dataset Customers(CustomerType) partitioned by key cid;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/tiny01/customer.adm"),("format"="adm"));
+
+create index age_index on Customers(age);
+
+write output to nc1:"rttest/index-selection_cust-index-age-nullable.adm";
+
+for $c in dataset('Customers')
+where $c.age < 20
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains.aql
new file mode 100644
index 0000000..ad85be7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/index-selection_inverted-index-ngram-contains.adm";
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic.aql
new file mode 100644
index 0000000..b275423
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/index-selection_inverted-index-ngram-edit-distance-panic.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance.aql
new file mode 100644
index 0000000..ddcdd4b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(authors) type ngram(3);
+
+write output to nc1:"rttest/index-selection_inverted-index-ngram-edit-distance.adm";
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard.aql
new file mode 100644
index 0000000..501ebce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index ngram_index on DBLP(title) type ngram(3);
+
+write output to nc1:"rttest/index-selection_inverted-index-ngram-jaccard.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
+where $jacc[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic.aql
new file mode 100644
index 0000000..deb51a7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic.aql
@@ -0,0 +1,37 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  partitioned by key cid on group1;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-olist-edit-distance-panic.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance.aql
new file mode 100644
index 0000000..bb05fc1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance.aql
@@ -0,0 +1,37 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  partitioned by key cid on group1;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-olist-edit-distance.adm";
+
+for $c in dataset('Customers')
+let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
+where $ed[0]
+order by $c.cid
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard.aql
new file mode 100644
index 0000000..8e2d1e7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard.aql
@@ -0,0 +1,36 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  partitioned by key cid on group1;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-olist-jaccard.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard.aql
new file mode 100644
index 0000000..6a0e266
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard.aql
@@ -0,0 +1,36 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  partitioned by key cid on group1;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
+
+create index interests_index on Customers(interests) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-ulist-jaccard.adm";
+
+for $c in dataset('Customers')
+let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
+where $jacc[0]
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains.aql
new file mode 100644
index 0000000..348f686
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-word-contains.adm";
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard.aql
new file mode 100644
index 0000000..9852e67
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+create index keyword_index on DBLP(title) type keyword;
+
+write output to nc1:"rttest/index-selection_inverted-index-word-jaccard.adm";
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open.aql
new file mode 100644
index 0000000..1a0ecbc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open.aql
@@ -0,0 +1,35 @@
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type OrderType as closed {
+  o_orderkey: int32, 
+  o_custkey: int32, 
+  o_orderstatus: string, 
+  o_totalprice: double, 
+  o_orderdate: string, 
+  o_orderpriority: string,
+  o_clerk: string, 
+  o_shippriority: int32, 
+  o_comment: string
+}
+
+create dataset Orders(OrderType)
+  partitioned by key o_orderkey;
+
+load dataset Orders 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_Orders_Custkey on Orders(o_custkey) ;
+
+write output to nc1:"rttest/index-selection_orders-index-custkey-conjunctive-open.adm";
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
+order by $o.o_orderkey
+return {  
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive.aql
new file mode 100644
index 0000000..ceca42e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive.aql
@@ -0,0 +1,35 @@
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type OrderType as closed {
+  o_orderkey: int32, 
+  o_custkey: int32, 
+  o_orderstatus: string, 
+  o_totalprice: double, 
+  o_orderdate: string, 
+  o_orderpriority: string,
+  o_clerk: string, 
+  o_shippriority: int32, 
+  o_comment: string
+}
+
+create dataset Orders(OrderType)
+  partitioned by key o_orderkey;
+
+load dataset Orders 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_Orders_Custkey on Orders(o_custkey) ;
+
+write output to nc1:"rttest/index-selection_orders-index-custkey-conjunctive.adm";
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
+order by $o.o_orderkey
+return {  
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open.aql
new file mode 100644
index 0000000..281f566
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open.aql
@@ -0,0 +1,38 @@
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type OrderType as open {
+  o_orderkey: int32, 
+  o_custkey: int32, 
+  o_orderstatus: string, 
+  o_totalprice: double, 
+  o_orderdate: string, 
+  o_orderpriority: string,
+  o_clerk: string, 
+  o_shippriority: int32, 
+  o_comment: string
+}
+
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset Orders(OrderType)
+  partitioned by key o_orderkey on group1;
+
+load dataset Orders 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
+
+write output to nc1:"rttest/index-selection_orders-index-custkey-open.adm";
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40
+order by $o.o_orderkey
+return {  
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey 
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey.aql
new file mode 100644
index 0000000..365cfcb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey.aql
@@ -0,0 +1,35 @@
+drop dataverse tpch if exists;
+create dataverse tpch;
+use dataverse tpch;
+
+create type OrderType as closed {
+  o_orderkey: int32, 
+  o_custkey: int32, 
+  o_orderstatus: string, 
+  o_totalprice: double, 
+  o_orderdate: string, 
+  o_orderpriority: string,
+  o_clerk: string, 
+  o_shippriority: int32, 
+  o_comment: string
+}
+
+create dataset Orders(OrderType)
+  partitioned by key o_orderkey;
+
+load dataset Orders 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
+
+write output to nc1:"rttest/index-selection_orders-index-custkey.adm";
+
+for $o in dataset('Orders')
+where
+  $o.o_custkey = 40
+order by $o.o_orderkey
+return {  
+  "o_orderkey": $o.o_orderkey,
+  "o_custkey": $o.o_custkey 
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open.aql
new file mode 100644
index 0000000..099e2d2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open.aql
@@ -0,0 +1,41 @@
+drop dataverse test if exists;
+  
+create dataverse test;
+use dataverse test;
+
+create type LineItemType as open {
+  l_orderkey: int32, 
+  l_partkey: int32, 
+  l_suppkey: int32, 
+  l_linenumber: int32, 
+  l_quantity: double, 
+  l_extendedprice: double,
+  l_discount: double, 
+  l_tax: double,
+  l_returnflag: string, 
+  l_linestatus: string, 
+  l_shipdate: string,
+  l_commitdate: string, 
+  l_receiptdate: string, 
+  l_shipinstruct: string, 
+  l_shipmode: string, 
+  l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+  partitioned by key l_orderkey, l_linenumber;
+
+load dataset LineItem 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_LineItem_partkey on LineItem(l_linenumber);
+create index idx_LineItem_suppkey on LineItem(l_suppkey);
+
+write output to nc1:"rttest/index-selection_range-search-open.adm";
+      
+for $c in dataset('LineItem')
+where $c.l_suppkey < 100 and $c.l_suppkey>5
+order by $c.l_orderkey, $c.l_linenumber
+return $c 
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search.aql
new file mode 100644
index 0000000..62714ed
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search.aql
@@ -0,0 +1,41 @@
+drop dataverse test if exists;
+  
+create dataverse test;
+use dataverse test;
+
+create type LineItemType as closed {
+  l_orderkey: int32, 
+  l_partkey: int32, 
+  l_suppkey: int32, 
+  l_linenumber: int32, 
+  l_quantity: double, 
+  l_extendedprice: double,
+  l_discount: double, 
+  l_tax: double,
+  l_returnflag: string, 
+  l_linestatus: string, 
+  l_shipdate: string,
+  l_commitdate: string, 
+  l_receiptdate: string, 
+  l_shipinstruct: string, 
+  l_shipmode: string, 
+  l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+  partitioned by key l_orderkey, l_linenumber;
+
+load dataset LineItem 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+create index idx_LineItem_partkey on LineItem(l_linenumber);
+create index idx_LineItem_suppkey on LineItem(l_suppkey);
+
+write output to nc1:"rttest/index-selection_range-search.adm";
+
+for $c in dataset('LineItem')
+where $c.l_suppkey < 100 and $c.l_suppkey>5
+order by $c.l_orderkey, $c.l_linenumber
+return $c 
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable.aql
new file mode 100644
index 0000000..6ca6d28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int32,
+  point: point?,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle
+}
+
+create dataset MyData(MyRecord)
+  partitioned by key id;
+
+load dataset MyData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/spatial/spatialDataNulls.json"),("format"="adm"));
+
+create index rtree_index_point on MyData(point) type rtree;
+
+write output to nc1:"rttest/index-selection_rtree-secondary-index-nullable.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
+order by $o.id
+return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open.aql
new file mode 100644
index 0000000..44425cc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open.aql
@@ -0,0 +1,31 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as open {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle
+}
+
+create dataset MyData(MyRecord)
+  partitioned by key id;
+
+load dataset MyData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/spatial/spatialData.json"),("format"="adm"));
+
+create index rtree_index_point on MyData(point) type rtree;
+
+
+write output to nc1:"rttest/index-selection_rtree-secondary-index-open.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
+order by $o.id
+return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index.aql
new file mode 100644
index 0000000..7ff775c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+  id: int32,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle
+}
+
+create dataset MyData(MyRecord)
+  partitioned by key id;
+
+load dataset MyData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
+(("path"="nc1://data/spatial/spatialData.json"),("format"="adm"));
+
+create index rtree_index_point on MyData(point) type rtree;
+
+write output to nc1:"rttest/index-selection_rtree-secondary-index.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
+order by $o.id
+return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/cust-index-age-nullable.aql b/asterix-app/src/test/resources/runtimets/queries/index/cust-index-age-nullable.aql
deleted file mode 100644
index 19348a6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/cust-index-age-nullable.aql
+++ /dev/null
@@ -1,33 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as open {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as open {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) partitioned by key cid;
-
-load dataset Customers 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/semistructured/tiny01/customer.adm"),("format"="adm"));
-
-create index age_index on Customers(age);
-
-write output to nc1:"rttest/index_cust-index-age-nullable.adm";
-
-for $c in dataset('Customers')
-where $c.age < 20
-order by $c.cid
-return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive-open.aql b/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive-open.aql
deleted file mode 100644
index 1a29b28..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive-open.aql
+++ /dev/null
@@ -1,35 +0,0 @@
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create dataset Orders(OrderType)
-  partitioned by key o_orderkey;
-
-load dataset Orders 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey) ;
-
-write output to nc1:"rttest/index_orders-index-custkey-conjunctive-open.adm";
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive.aql b/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive.aql
deleted file mode 100644
index 10db179..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-conjunctive.aql
+++ /dev/null
@@ -1,35 +0,0 @@
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create dataset Orders(OrderType)
-  partitioned by key o_orderkey;
-
-load dataset Orders 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey) ;
-
-write output to nc1:"rttest/index_orders-index-custkey-conjunctive.adm";
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-open.aql b/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-open.aql
deleted file mode 100644
index f3a9f1f..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey-open.aql
+++ /dev/null
@@ -1,38 +0,0 @@
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on nc1, nc2;
-
-create dataset Orders(OrderType)
-  partitioned by key o_orderkey on group1;
-
-load dataset Orders 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to nc1:"rttest/index_orders-index-custkey-open.adm";
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey 
-}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey.aql b/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey.aql
deleted file mode 100644
index 38bc76c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/orders-index-custkey.aql
+++ /dev/null
@@ -1,35 +0,0 @@
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create dataset Orders(OrderType)
-  partitioned by key o_orderkey;
-
-load dataset Orders 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to nc1:"rttest/index_orders-index-custkey.adm";
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey 
-}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/range-search-open.aql b/asterix-app/src/test/resources/runtimets/queries/index/range-search-open.aql
deleted file mode 100644
index 1781280..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/range-search-open.aql
+++ /dev/null
@@ -1,40 +0,0 @@
-drop dataverse test if exists;
-  
-create dataverse test;
-use dataverse test;
-
-create type LineItemType as open {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  partitioned by key l_orderkey, l_linenumber;
-
-load dataset LineItem 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_LineItem_partkey on LineItem(l_linenumber);
-create index idx_LineItem_suppkey on LineItem(l_suppkey);
-
-write output to nc1:"rttest/index_range-search-open.adm";      
-for $c in dataset('LineItem')
-where $c.l_suppkey < 100 and $c.l_suppkey>5
-order by $c.l_orderkey, $c.l_linenumber
-return $c 
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/range-search.aql b/asterix-app/src/test/resources/runtimets/queries/index/range-search.aql
deleted file mode 100644
index 666dc73..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/range-search.aql
+++ /dev/null
@@ -1,40 +0,0 @@
-drop dataverse test if exists;
-  
-create dataverse test;
-use dataverse test;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  partitioned by key l_orderkey, l_linenumber;
-
-load dataset LineItem 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_LineItem_partkey on LineItem(l_linenumber);
-create index idx_LineItem_suppkey on LineItem(l_suppkey);
-
-write output to nc1:"rttest/index_range-search.adm";      
-for $c in dataset('LineItem')
-where $c.l_suppkey < 100 and $c.l_suppkey>5
-order by $c.l_orderkey, $c.l_linenumber
-return $c 
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-nullable.aql b/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-nullable.aql
deleted file mode 100644
index a4555f4..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-nullable.aql
+++ /dev/null
@@ -1,30 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point?,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData(MyRecord)
-  partitioned by key id;
-
-load dataset MyData 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/spatial/spatialDataNulls.json"),("format"="adm"));
-
-create index rtree_index_point on MyData(point) type rtree;
-
-write output to nc1:"rttest/index_rtree-secondary-index-nullable.adm";
-
-for $o in dataset('MyData')
-where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
-order by $o.id
-return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-open.aql b/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-open.aql
deleted file mode 100644
index c428af2..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index-open.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as open {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData(MyRecord)
-  partitioned by key id;
-
-load dataset MyData 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/spatial/spatialData.json"),("format"="adm"));
-
-create index rtree_index_point on MyData(point) type rtree;
-
-
-write output to nc1:"rttest/index_rtree-secondary-index-open.adm";
-
-for $o in dataset('MyData')
-where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
-order by $o.id
-return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index.aql b/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index.aql
deleted file mode 100644
index 01b2981..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index/rtree-secondary-index.aql
+++ /dev/null
@@ -1,30 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData(MyRecord)
-  partitioned by key id;
-
-load dataset MyData 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter" 
-(("path"="nc1://data/spatial/spatialData.json"),("format"="adm"));
-
-create index rtree_index_point on MyData(point) type rtree;
-
-write output to nc1:"rttest/index_rtree-secondary-index.adm";
-
-for $o in dataset('MyData')
-where spatial-intersect($o.point, create-polygon(create-point(4.0,1.0), create-point(4.0,4.0), create-point(12.0,4.0), create-point(12.0,1.0)))
-order by $o.id
-return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/abs0.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/abs0.aql
new file mode 100644
index 0000000..3908231
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/abs0.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_abs0.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-abs($c0), "f1": numeric-abs($c1),"f2": numeric-abs($c2), "f3": numeric-abs($c3),
+	"f4": numeric-abs($c4),"f5": numeric-abs($c5) ,"f6": numeric-abs($c6), "f7": numeric-abs($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/abs1.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/abs1.aql
new file mode 100644
index 0000000..9d36528
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/abs1.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_abs1.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-abs($c0), "f1": numeric-abs($c1),"f2": numeric-abs($c2), "f3": numeric-abs($c3),
+	"f4": numeric-abs($c4),"f5": numeric-abs($c5) ,"f6": numeric-abs($c6), "f7": numeric-abs($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/abs2.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/abs2.aql
new file mode 100644
index 0000000..411ae00
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/abs2.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_abs2.adm";
+
+
+let $c0 := float("-20.1")
+let $c1 := float("-20.56e-30")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"f0": numeric-abs($c0), "f1": numeric-abs($c1),"f2": numeric-abs($c2),
+        "f3": numeric-abs($c3),"f4": numeric-abs($c4),"f5": numeric-abs($c5), "f6": numeric-abs($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/abs3.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/abs3.aql
new file mode 100644
index 0000000..4364409
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/abs3.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_abs3.adm";
+
+
+let $c0 := double("-20.1")
+let $c1 := double("-20.56e-30")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-abs($c0), "d1": numeric-abs($c1),"d2": numeric-abs($c2),
+        "d3": numeric-abs($c3),"d4": numeric-abs($c4),"d5": numeric-abs($c5), "d6": numeric-abs($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/abs4.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/abs4.aql
new file mode 100644
index 0000000..b2db5fd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/abs4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_abs4.adm";
+
+let $c0 := int8("-20")
+return {"f0": numeric-abs($c0), "f1": numeric-abs(-1.11),"f2": numeric-abs(12.9), "f3": numeric-abs(1.11)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling0.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling0.aql
new file mode 100644
index 0000000..82e314f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling0.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_ceiling0.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-ceiling($c0), "f1": numeric-ceiling($c1),"f2": numeric-ceiling($c2), "f3": numeric-ceiling($c3),
+	"f4": numeric-ceiling($c4),"f5": numeric-ceiling($c5) ,"f6": numeric-ceiling($c6), "f7": numeric-ceiling($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling1.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling1.aql
new file mode 100644
index 0000000..88e8d63
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling1.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_ceiling1.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-ceiling($c0), "f1": numeric-ceiling($c1),"f2": numeric-ceiling($c2), "f3": numeric-ceiling($c3),
+	"f4": numeric-ceiling($c4),"f5": numeric-ceiling($c5) ,"f6": numeric-ceiling($c6), "f7": numeric-ceiling($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling2.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling2.aql
new file mode 100644
index 0000000..8f71213
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling2.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_ceiling2.adm";
+
+
+let $c0 := float("20.1")
+let $c1 := float("-20.56e-30")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"f0": numeric-ceiling($c0), "f1": numeric-ceiling($c1),"f2": numeric-ceiling($c2),
+        "f3": numeric-ceiling($c3),"f4": numeric-ceiling($c4),"f5": numeric-ceiling($c5), "f6": numeric-ceiling($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling3.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling3.aql
new file mode 100644
index 0000000..67ad9db
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling3.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_ceiling3.adm";
+
+
+let $c0 := double("20.1")
+let $c1 := double("-20.56e-30")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-ceiling($c0), "d1": numeric-ceiling($c1),"d2": numeric-ceiling($c2),
+        "d3": numeric-ceiling($c3),"d4": numeric-ceiling($c4),"d5": numeric-ceiling($c5), "d6": numeric-ceiling($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling4.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling4.aql
new file mode 100644
index 0000000..a90851c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/ceiling4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_ceiling4.adm";
+
+let $c0 := int8("-20")
+return {"f0": numeric-ceiling($c0), "f1": numeric-ceiling(-1.11),"f2": numeric-ceiling(12.9), "f3": numeric-ceiling(1.11)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/floor0.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/floor0.aql
new file mode 100644
index 0000000..adfab71
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/floor0.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_floor0.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-floor($c0), "f1": numeric-floor($c1),"f2": numeric-floor($c2), "f3": numeric-floor($c3),
+	"f4": numeric-floor($c4),"f5": numeric-floor($c5) ,"f6": numeric-floor($c6), "f7": numeric-floor($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/floor1.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/floor1.aql
new file mode 100644
index 0000000..a119b67
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/floor1.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_floor1.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-floor($c0), "f1": numeric-floor($c1),"f2": numeric-floor($c2), "f3": numeric-floor($c3),
+	"f4": numeric-floor($c4),"f5": numeric-floor($c5) ,"f6": numeric-floor($c6), "f7": numeric-floor($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/floor2.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/floor2.aql
new file mode 100644
index 0000000..7e575dd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/floor2.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_floor2.adm";
+
+
+let $c0 := float("20.1")
+let $c1 := float("-20.56e-30")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"f0": numeric-floor($c0), "f1": numeric-floor($c1),"f2": numeric-floor($c2),
+        "f3": numeric-floor($c3),"f4": numeric-floor($c4),"f5": numeric-floor($c5), "f6": numeric-floor($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/floor3.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/floor3.aql
new file mode 100644
index 0000000..d975dfc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/floor3.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_floor3.adm";
+
+
+let $c0 := double("20.1")
+let $c1 := double("-20.56e-30")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-floor($c0), "d1": numeric-floor($c1),"d2": numeric-floor($c2),
+        "d3": numeric-floor($c3),"d4": numeric-floor($c4),"d5": numeric-floor($c5), "d6": numeric-floor($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/floor4.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/floor4.aql
new file mode 100644
index 0000000..eae917d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/floor4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_floor4.adm";
+
+let $c0 := int8("-20")
+return {"f0": numeric-floor($c0), "f1": numeric-floor(-1.11),"f2": numeric-floor(12.9), "f3": numeric-floor(1.11)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even0.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even0.aql
new file mode 100644
index 0000000..25232d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even0.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even0.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-round-half-to-even($c0), "f1": numeric-round-half-to-even($c1),"f2": numeric-round-half-to-even($c2), "f3": numeric-round-half-to-even($c3),
+	"f4": numeric-round-half-to-even($c4),"f5": numeric-round-half-to-even($c5) ,"f6": numeric-round-half-to-even($c6), "f7": numeric-round-half-to-even($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even1.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even1.aql
new file mode 100644
index 0000000..eeccdd7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even1.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even1.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-round-half-to-even($c0), "f1": numeric-round-half-to-even($c1),"f2": numeric-round-half-to-even($c2), "f3": numeric-round-half-to-even($c3),
+	"f4": numeric-round-half-to-even($c4),"f5": numeric-round-half-to-even($c5) ,"f6": numeric-round-half-to-even($c6), "f7": numeric-round-half-to-even($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even2.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even2.aql
new file mode 100644
index 0000000..8b339ba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even2.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even2.adm";
+
+
+let $c0 := float("0.5")
+let $c1 := float("-20.5")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"f0": numeric-round-half-to-even($c0), "f1": numeric-round-half-to-even($c1),"f2": numeric-round-half-to-even($c2),
+        "f3": numeric-round-half-to-even($c3),"f4": numeric-round-half-to-even($c4),"f5": numeric-round-half-to-even($c5), "f6": numeric-round-half-to-even($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20.aql
new file mode 100644
index 0000000..92088b4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even20.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-round-half-to-even2($c0,2), "f1": numeric-round-half-to-even2($c1,2),"f2": numeric-round-half-to-even2($c2,2), "f3": numeric-round-half-to-even2($c3,2),
+	"f4": numeric-round-half-to-even2($c4,2),"f5": numeric-round-half-to-even2($c5,2) ,"f6": numeric-round-half-to-even2($c6,2), "f7": numeric-round-half-to-even2($c7,2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21.aql
new file mode 100644
index 0000000..c00cc25
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even21.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-round-half-to-even2($c0,2), "f1": numeric-round-half-to-even2($c1,2),"f2": numeric-round-half-to-even2($c2,2), "f3": numeric-round-half-to-even2($c3,2),
+	"f4": numeric-round-half-to-even2($c4,2),"f5": numeric-round-half-to-even2($c5,2) ,"f6": numeric-round-half-to-even2($c6,2), "f7": numeric-round-half-to-even2($c7,2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22.aql
new file mode 100644
index 0000000..22ff5ad
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even22.adm";
+
+
+let $c0 := float("0.555")
+let $c1 := float("0.322")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,3),
+        "d3": numeric-round-half-to-even2($c3,4),"d4": numeric-round-half-to-even2($c4,5),"d5": numeric-round-half-to-even2($c5,6), "d6": numeric-round-half-to-even2($c6,0)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23.aql
new file mode 100644
index 0000000..cbbcd10
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even23.adm";
+
+
+let $c0 := double("0.555")
+let $c1 := double("0.322")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,3),
+        "d3": numeric-round-half-to-even2($c3,4),"d4": numeric-round-half-to-even2($c4,5),"d5": numeric-round-half-to-even2($c5,6), "d6": numeric-round-half-to-even2($c6,0)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24.aql
new file mode 100644
index 0000000..8fe21fe
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24.aql
@@ -0,0 +1,14 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even24.adm";
+
+
+let $c0 := double("0.015")
+let $c1 := double("0.025")
+let $c2 := double("3.567812E+3")
+let $c3 := double("4.7564E-3")
+let $c4 := double("35612.25")
+return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,2),
+        "d3": numeric-round-half-to-even2($c3,2),"d4": numeric-round-half-to-even2($c4,-2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even3.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even3.aql
new file mode 100644
index 0000000..6d10ae1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even3.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even3.adm";
+
+
+let $c0 := double("0.5")
+let $c1 := double("-20.5")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-round-half-to-even($c0), "d1": numeric-round-half-to-even($c1), "d2": numeric-round-half-to-even($c2),
+        "d3": numeric-round-half-to-even($c3), "d4": numeric-round-half-to-even($c4), "d5": numeric-round-half-to-even($c5), "d6": numeric-round-half-to-even($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even4.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even4.aql
new file mode 100644
index 0000000..3c24b0b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even4.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even4.adm";
+
+
+let $c0 := double("1.5")
+let $c1 := double("2.5")
+return {"d0": numeric-round-half-to-even($c0), "d1": numeric-round-half-to-even($c1)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even5.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even5.aql
new file mode 100644
index 0000000..b89388f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even5.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round-half-to-even5.adm";
+
+let $c0 := int8("-20")
+return {"f0": numeric-round-half-to-even($c0), "f1": numeric-round-half-to-even(-1.5),"f2": numeric-round-half-to-even(12.5), "f3": numeric-round-half-to-even(1.5)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round0.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round0.aql
new file mode 100644
index 0000000..37887e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round0.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round0.adm";
+
+
+let $c0 := int8("-0")
+let $c1 := int16("-0")
+let $c2 := int32("-0")
+let $c3 := int64("-0")
+let $c4 := int8("0")
+let $c5 := int16("0")
+let $c6 := int32("0")
+let $c7 := int64("0")
+return {"f0": numeric-round($c0), "f1": numeric-round($c1),"f2": numeric-round($c2), "f3": numeric-round($c3),
+	"f4": numeric-round($c4),"f5": numeric-round($c5) ,"f6": numeric-round($c6), "f7": numeric-round($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round1.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round1.aql
new file mode 100644
index 0000000..5c25bf8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round1.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round1.adm";
+
+
+let $c0 := int8("-20")
+let $c1 := int16("-23")
+let $c2 := int32("-29")
+let $c3 := int64("-21")
+let $c4 := int8("20")
+let $c5 := int16("22")
+let $c6 := int32("23")
+let $c7 := int64("27")
+return {"f0": numeric-round($c0), "f1": numeric-round($c1),"f2": numeric-round($c2), "f3": numeric-round($c3),
+	"f4": numeric-round($c4),"f5": numeric-round($c5) ,"f6": numeric-round($c6), "f7": numeric-round($c7)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round2.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round2.aql
new file mode 100644
index 0000000..e5e75bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round2.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round2.adm";
+
+
+let $c0 := float("20.1")
+let $c1 := float("-20.56e-30")
+let $c2 := float("NaN")
+let $c3 := float("INF")
+let $c4 := float("-INF")
+let $c5 := float("-0.0")
+let $c6 := float("0.0")
+return {"f0": numeric-round($c0), "f1": numeric-round($c1),"f2": numeric-round($c2),
+        "f3": numeric-round($c3),"f4": numeric-round($c4),"f5": numeric-round($c5), "f6": numeric-round($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round3.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round3.aql
new file mode 100644
index 0000000..54e2685
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round3.aql
@@ -0,0 +1,16 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round3.adm";
+
+
+let $c0 := double("20.1")
+let $c1 := double("-20.56e-30")
+let $c2 := double("NaN")
+let $c3 := double("INF")
+let $c4 := double("-INF")
+let $c5 := double("-0.0")
+let $c6 := double("0.0")
+return {"d0": numeric-round($c0), "d1": numeric-round($c1),"d2": numeric-round($c2),
+        "d3": numeric-round($c3),"d4": numeric-round($c4),"d5": numeric-round($c5), "d6": numeric-round($c6)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round4.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round4.aql
new file mode 100644
index 0000000..5dccfd7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/numeric_round4.adm";
+
+let $c0 := int8("-20")
+return {"f0": numeric-round($c0), "f1": numeric-round(-1.11),"f2": numeric-round(12.9), "f3": numeric-round(1.11)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_02.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_02.aql
new file mode 100644
index 0000000..fad06ba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_02.aql
@@ -0,0 +1,29 @@
+/*
+ * Description      : Test Quantified Expressions 
+ *                  : every <variable-name> in [ordered-list], <variable-name> in [ordered-list] satisfies expression
+ * Expected Result  : Success
+ * Date             : 5th July 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/quantifiers_everysat_02.adm";
+
+let $a := [
+every $x in [1, 2] satisfies $x + $x = 3,
+every $x in [1, 1] satisfies $x + $x = 2,
+every $x in [1, 2] satisfies $x - 2 = 2,
+every $x in [2, 2] satisfies $x - 2 = 0,
+every $x in [1, 2] satisfies $x * 2 = 4,
+every $x in [1, 2] satisfies $x / 2 = 1,
+every $x in [1, 2] satisfies $x = 1 or $x = 2,
+every $x in [1, 2] satisfies $x = 1 and ($x +1) = 2,
+every $x in ["A","B","C"] satisfies $x = "A",
+every $x in [1,2,3], $y in [4,5,6] satisfies $x + $y = 5,
+every $x in [1,2,3], $y in [4,5,6] satisfies $x - $y = 5,
+every $x in [1,2,3], $y in [4,5,6] satisfies $x * $y = 10
+]
+for $i in $a
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_03.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_03.aql
new file mode 100644
index 0000000..7f9824e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_03.aql
@@ -0,0 +1,33 @@
+/*
+ * Description      : Test Quantified Expressions 
+ *                  : every <variable-name> in [ordered-list], <variable-name> in [ordered-list] satisfies expression
+ * Expected Result  : Success
+ * Date             : 5th July 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/quantifiers_everysat_02.adm";
+
+let $a := [
+every $x in [1, 2] satisfies avg([$x, 1]) = 1,
+every $x in [1, 2] satisfies string($x) = "1",
+every $x in [1, 2] satisfies string-length(string($x)) = 1,
+every $x in [[1, 2],[10],[1,5,7,8]] satisfies count($x) = 1,
+every $x in [[2],[10],[8]] satisfies count($x) = 1,
+every $x in [1, 2] satisfies boolean("true"),
+every $x in [1, 2] satisfies not($x),
+every $x in [1,2,3], $y in [4,5,6] satisfies $x + $y = 5,
+every $x in [1,2,3], $y in [4,5,6] satisfies $x - $y = 5,
+every $x in [1,2,3], $y in [4,5,6] satisfies $x * $y = 10,
+every $x in [1,2,3], $y in [4,5,6] satisfies string($x) = string($y),
+every $x in [1,2,3], $y in [4,5,6] satisfies int32($x) = int32($y),
+every $x in [1,2,3], $y in [4,5,6] satisfies float($x) = float($y),
+every $x in [1,2,3], $y in [4,5,6] satisfies double($x) = double($y),
+every $x in ["true", "false"], $y in ["false","true"] satisfies boolean($x) = boolean($y),
+every $x in ["1980-05-05T13:13:13Z", "1980-05-05T13:13:13Z"], $y in ["1980-05-05T13:13:13Z","1980-05-05T13:13:13Z"] satisfies datetime($x) = datetime($y)
+]
+for $i in $a
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_03.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_03.aql
new file mode 100644
index 0000000..5995b9b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_03.aql
@@ -0,0 +1,30 @@
+/*
+ * Description     :  Test quantified expressions; some variable in [ordered list] satisfies expression.
+ * Expected Result : Success
+ * Date            : 6th July 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/quantifiers_somesat_03.adm";
+
+let $a := [
+some $x in [1, 2] satisfies $x + $x = 3,
+some $x in [1, 2] satisfies $x + $x = 2,
+some $x in [1, 2] satisfies $x - 2 = 2,
+some $x in [1, 2] satisfies $x - 2 = 0,
+some $x in [1, 2] satisfies $x * 2 = 4,
+some $x in [1, 2] satisfies $x / 2 = 1,
+some $x in [1, 2] satisfies avg([$x,1]) = 1,
+some $x in [1, 2] satisfies boolean("true"),
+some $x in [1, 2] satisfies boolean("false"),
+some $x in [1, 2] satisfies not($x),
+some $x in [1, 2] satisfies $x = 1 or $x = 2,
+some $x in [1, 2] satisfies $x = 1 and ($x +1) = 2 
+]
+for $i in $a
+return $i
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_04.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_04.aql
new file mode 100644
index 0000000..8d92cbb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_04.aql
@@ -0,0 +1,28 @@
+/*
+ * Description      : Test Quantified Expressions
+ *                  : some <variable-name> in [ordered-list] satisfies function expression
+ *                  : some <variable-name> in [ordered-list],<variable-name> in [ordered-list] satisfies expression
+ * Expected Result  : Success
+ * Date             : 5th July 2012
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/quantifiers_somesat_04.adm";
+
+let $a := [
+some $x in ["foo","foobar","foot","fox"] satisfies string-length($x) = 1,
+some $x in [1,2,3,4,5,6,7,8] satisfies count($x) = 8,
+some $x in [1, 2] satisfies $x = 1 or $x = 2,
+some $x in [1, 2] satisfies $x = 1 and ($x +1) = 2,
+some $x in ["A","B","C"] satisfies $x = "A",
+some $x in [1,2,3], $y in [4,5,6] satisfies $x + $y = 5,
+some $x in [1,2,3], $y in [4,5,6] satisfies $x - $y = 5,
+some $x in [1,2,3], $y in [4,5,6] satisfies $x * $y = 10,
+some $x in [1,2,3], $y in [4,5,6] satisfies $x / $y = 2
+]
+for $i in $a
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_05.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_05.aql
new file mode 100644
index 0000000..fefd86d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_05.aql
@@ -0,0 +1,24 @@
+/*
+ * Description      :  Test Quantified Expressions 
+ *                  :  some <variable-name> in [ordered-list], <variable-name> in [ordered-list] satisfies function expression
+ * Expected Result  :  Success
+ * Date             :  5th July 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/quantifiers_somesat_05.adm";
+
+let $a := [
+some $x in ["foo","foobar","footnote"], $y in ["foofoo","fool","foolish","foot","foo"] satisfies string($x) = string($y),
+some $x in ["1","2","3"], $y in ["4","5","6"] satisfies int32($x) = int32($y),
+some $x in ["1.1","2.2","3.3"], $y in ["4.4","5.5","6.6"] satisfies float($x) = float($y),
+some $x in ["1.1d","2.2d","3.3d"], $y in ["4.4d","5.5d","6.6d"] satisfies double($x) = double($y),
+some $x in ["true", "false"], $y in ["false","true"] satisfies boolean($x) = boolean($y),
+some $x in ["1980-05-05T13:13:13Z", "1980-05-05T13:13:13Z"], $y in ["1980-05-05T13:13:13Z","1980-05-05T13:13:13Z"] satisfies datetime($x) = datetime($y),
+some $x in ["1985-07-05Z", "1985-07-05Z"], $y in ["1985-07-05Z","1985-07-05Z"] satisfies date($x) = date($y)
+]
+for $i in $a
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_01.aql
deleted file mode 100644
index 2034c38..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_01.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance-check_01.adm";
-
-let $a := "Nalini Venkatasubramanian"
-let $b := "Nalini Wekatasupramanian"
-let $ed := edit-distance-check($a, $b, 3)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_02.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_02.aql
deleted file mode 100644
index c3d5342..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_02.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance-check_02.adm";
-
-let $a := "Nalini Venkatasubramanian"
-let $b := "Nalini Wekatasupramanian"
-let $ed := edit-distance-check($a, $b, 2)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_03.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_03.aql
deleted file mode 100644
index ecf556c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_03.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance-check_03.adm";
-
-let $a := [1, 2, 3, 4, 5, 6, 7]
-let $b := [1, 3, 4, 5, 7, 8]
-let $ed := edit-distance-check($a, $b, 3)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_04.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_04.aql
deleted file mode 100644
index 80a2da6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_04.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance-check_04.adm";
-
-let $a := [1, 2, 3, 4, 5, 6, 7]
-let $b := [1, 3, 4, 5, 7, 8]
-let $ed := edit-distance-check($a, $b, 2)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_ints.aql
new file mode 100644
index 0000000..3b5fd8d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_ints.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance-check_ints.adm";
+
+let $a := [1, 2, 3, 4, 5, 6, 7]
+let $b := [1, 3, 4, 5, 7, 8]
+let $results :=
+[
+  edit-distance-check($a, $b, 3),
+  edit-distance-check($b, $a, 3),
+  edit-distance-check($a, $b, 2),
+  edit-distance-check($b, $a, 2)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_strings.aql
new file mode 100644
index 0000000..e861679
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-check_strings.aql
@@ -0,0 +1,17 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance-check_strings.adm";
+
+let $a := "Nalini Venkatasubramanian"
+let $b := "Nalini Wekatasupramanian"
+let $results :=
+[
+  edit-distance-check($a, $b, 3),
+  edit-distance-check($b, $a, 3),
+  edit-distance-check($a, $b, 2),
+  edit-distance-check($b, $a, 2)
+]
+for $i in $results
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable.aql
new file mode 100644
index 0000000..ecfbd52
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable.aql
@@ -0,0 +1,19 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance-list-is-filterable.adm";
+
+let $a := []
+let $b := [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]
+let $results :=
+[
+  edit-distance-list-is-filterable($a, 0),
+  edit-distance-list-is-filterable($a, 3),
+  edit-distance-list-is-filterable($b, 0),
+  edit-distance-list-is-filterable($b, 3),
+  edit-distance-list-is-filterable($b, 8),
+  edit-distance-list-is-filterable($b, 11)
+]
+for $i in $results
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable.aql
new file mode 100644
index 0000000..c306e0c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable.aql
@@ -0,0 +1,33 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance-string-is-filterable.adm";
+
+let $a := ""
+let $b := "abcdefghij"
+let $results :=
+[
+  edit-distance-string-is-filterable($a, 0, 2, false),
+  edit-distance-string-is-filterable($a, 0, 2, true),
+  edit-distance-string-is-filterable($a, 1, 2, false),
+  edit-distance-string-is-filterable($a, 1, 2, true),
+  edit-distance-string-is-filterable($b, 0, 2, false),
+  edit-distance-string-is-filterable($b, 0, 2, true),
+  edit-distance-string-is-filterable($b, 1, 2, false),
+  edit-distance-string-is-filterable($b, 1, 2, true),    
+  edit-distance-string-is-filterable($b, 4, 2, false),
+  edit-distance-string-is-filterable($b, 5, 2, true), 
+  edit-distance-string-is-filterable($b, 5, 2, false),
+  edit-distance-string-is-filterable($b, 6, 2, true),
+  edit-distance-string-is-filterable($b, 0, 3, false),
+  edit-distance-string-is-filterable($b, 0, 3, true),
+  edit-distance-string-is-filterable($b, 1, 3, false),
+  edit-distance-string-is-filterable($b, 1, 3, true),
+  edit-distance-string-is-filterable($b, 2, 3, false),
+  edit-distance-string-is-filterable($b, 3, 3, true), 
+  edit-distance-string-is-filterable($b, 3, 3, false),
+  edit-distance-string-is-filterable($b, 4, 3, true)  
+]
+for $i in $results
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_01.aql
deleted file mode 100644
index 3365690..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_01.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance_01.adm";
-
-let $a := "Nalini Venkatasubramanian"
-let $b := "Nalini Wekatasupramanian"
-let $ed := edit-distance($a, $b)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_02.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_02.aql
deleted file mode 100644
index bf0df90..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_02.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_edit-distance_02.adm";
-
-let $a := [1, 2, 3, 4, 5, 6, 7]
-let $b := [1, 3, 4, 5, 7, 8]
-let $ed := edit-distance($a, $b)
-return $ed
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_ints.aql
new file mode 100644
index 0000000..9cd7dc6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_ints.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance_ints.adm";
+
+let $a := [1, 2, 3, 4, 5, 6, 7]
+let $b := [1, 3, 4, 5, 7, 8]
+let $results :=
+[
+  edit-distance($a, $b),
+  edit-distance($b, $a)  
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_strings.aql
new file mode 100644
index 0000000..a721c40
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance_strings.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_edit-distance_strings.adm";
+
+let $a := "Nalini Venkatasubramanian"
+let $b := "Nalini Wekatasupramanian"
+let $results :=
+[
+  edit-distance($a, $b),
+  edit-distance($b, $a)  
+]
+for $i in $results
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-edit-distance.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-edit-distance.aql
new file mode 100644
index 0000000..d3eed71
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-edit-distance.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_fuzzyeq-edit-distance.adm";
+
+set simfunction 'edit-distance';
+set simthreshold '2';
+
+for $paper in dataset('DBLP')
+where $paper.authors ~= "Amihay Motro"
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-similarity-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-similarity-jaccard.aql
new file mode 100644
index 0000000..0c11edb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/fuzzyeq-similarity-jaccard.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_fuzzyeq-similarity-jaccard.adm";
+
+set simfunction 'jaccard';
+set simthreshold '0.5f';
+
+for $paper in dataset('DBLP')
+where word-tokens($paper.title) ~= word-tokens("Transactions for Cooperative Environments")
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard.aql
new file mode 100644
index 0000000..63424f4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_prefix-len-jaccard.adm";
+
+for $l in [1]
+return [
+  prefix-len-jaccard(5, .8f),
+  prefix-len-jaccard(5, .9f),
+  prefix-len-jaccard(10, .8f),
+  prefix-len-jaccard(10, .9f),
+  prefix-len-jaccard(15, .8f),
+  prefix-len-jaccard(15, .9f)
+]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard_01.aql
deleted file mode 100644
index 772e64e..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard_01.aql
+++ /dev/null
@@ -1,15 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_prefix-len-jaccard_01.adm";
-
-for $l in [1]
-return [
-  prefix-len-jaccard(5, .8f),
-  prefix-len-jaccard(5, .9f),
-  prefix-len-jaccard(10, .8f),
-  prefix-len-jaccard(10, .9f),
-  prefix-len-jaccard(15, .8f),
-  prefix-len-jaccard(15, .9f)
-]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_01.aql
deleted file mode 100644
index cca0f99..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_01.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard-check_01.adm";
-
-let $a := [1, 2, 3, 4, 5, 8, 9]
-let $b := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
-let $jacc := similarity-jaccard-check($a, $b, 0.7f)
-return $jacc
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_02.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_02.aql
deleted file mode 100644
index 38fe1f4..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_02.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard-check_02.adm";
-
-let $a := [1, 2, 3, 4, 5, 8, 9]
-let $b := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
-let $jacc := similarity-jaccard-check($a, $b, 0.8f)
-return $jacc
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_ints.aql
new file mode 100644
index 0000000..0791f0c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_ints.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-check_ints.adm";
+
+let $a := [ ]
+let $b := [1, 2, 3, 4, 5]
+let $c := [4, 3, 5, 8, 9, 2, 1]
+let $d := [7, 5, 8, 9, 3, 10, 1, 2, 11, 4]
+let $e := [11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21]
+let $f := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
+let $results :=
+[ 
+  similarity-jaccard-check($a, $b, 0.0f),
+  similarity-jaccard-check($b, $a, 0.0f),
+  similarity-jaccard-check($a, $b, 0.1f),
+  similarity-jaccard-check($b, $a, 0.1f),
+  similarity-jaccard-check($c, $d, 0.6f),
+  similarity-jaccard-check($d, $c, 0.6f),
+  similarity-jaccard-check($c, $d, 0.8f),
+  similarity-jaccard-check($d, $c, 0.8f),
+  similarity-jaccard-check($e, $f, 0.05f),
+  similarity-jaccard-check($f, $e, 0.05f),
+  similarity-jaccard-check($e, $f, 0.8f),
+  similarity-jaccard-check($f, $e, 0.8f)
+  
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_query.aql
new file mode 100644
index 0000000..01bea0b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_query.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-check_query.adm";
+
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Environments for Cooperative Transactions")
+let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
+where $jacc[0]
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings.aql
new file mode 100644
index 0000000..dadca7b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings.aql
@@ -0,0 +1,35 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-check_strings.adm";
+
+let $a := [ ]
+let $b := ["abc", "bcd", "cde", "def", "efg"]
+let $c := ["efg", "abc", "cde", "def", "hij", "ijk", "bcd"]
+let $d := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $e := ["Efg", "aBc", "cdE", "DEf", "hIJ", "IjK", "BCD"]
+let $f := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $g := ["cde", "zza", "zzb", "zzc", "zwz", "za", "zbe", "zer", "zba", "zfe", "wab"]
+let $h := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $results :=
+[ 
+  similarity-jaccard-check($a, $b, 0.0f),
+  similarity-jaccard-check($b, $a, 0.0f),
+  similarity-jaccard-check($a, $b, 0.1f),
+  similarity-jaccard-check($b, $a, 0.1f),
+  similarity-jaccard-check($c, $d, 0.6f),
+  similarity-jaccard-check($d, $c, 0.6f),
+  similarity-jaccard-check($c, $d, 0.8f),
+  similarity-jaccard-check($d, $c, 0.8f),
+  similarity-jaccard-check($e, $f, 0.6f),
+  similarity-jaccard-check($f, $e, 0.6f),
+  similarity-jaccard-check($e, $f, 0.8f),
+  similarity-jaccard-check($f, $e, 0.8f),
+  similarity-jaccard-check($g, $h, 0.05f),
+  similarity-jaccard-check($h, $g, 0.05f),
+  similarity-jaccard-check($g, $h, 0.8f),
+  similarity-jaccard-check($h, $g, 0.8f)  
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check.aql
new file mode 100644
index 0000000..7a48854
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-prefix-check.adm";
+
+for $l in [1]
+return [
+  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 3], 1, 1f),
+  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 4], 1, .5f),
+  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 4], 1, .6f),
+  similarity-jaccard-prefix-check(3, [1, 2, 3], 9, [1, 2, 3], 1, .5f),
+  similarity-jaccard-prefix-check(4, [1, 2, 3, 4], 2, [1, 2], 1, .5f),
+  similarity-jaccard-prefix-check(4, [1, 2, 3, 4], 4, [1, 2], 1, .33f)
+]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check_01.aql
deleted file mode 100644
index 73c7ebc..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check_01.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard-prefix-check_01.adm";
-
-for $l in [1]
-return [
-  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 3], 1, 1f),
-
-  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 4], 1, .5f),
-  similarity-jaccard-prefix-check(3, [1, 2, 3], 3, [1, 2, 4], 1, .6f),
-
-
-  similarity-jaccard-prefix-check(3, [1, 2, 3], 9, [1, 2, 3], 1, .5f),
-
-  similarity-jaccard-prefix-check(4, [1, 2, 3, 4], 2, [1, 2], 1, .5f),
-  similarity-jaccard-prefix-check(4, [1, 2, 3, 4], 4, [1, 2], 1, .33f)
-]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix.aql
new file mode 100644
index 0000000..50c857b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-prefix.adm";
+
+for $l in [1]
+return [
+  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 3], 1, 1f),
+  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 4], 1, .5f),
+  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 4], 1, .6f),
+  similarity-jaccard-prefix(3, [1, 2, 3], 9, [1, 2, 3], 1, .5f),
+  similarity-jaccard-prefix(4, [1, 2, 3, 4], 2, [1, 2], 1, .5f),
+  similarity-jaccard-prefix(4, [1, 2, 3, 4], 4, [1, 2], 1, .33f)
+]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix_01.aql
deleted file mode 100644
index 35b4719..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix_01.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard-prefix_01.adm";
-
-for $l in [1]
-return [
-  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 3], 1, 1f),
-
-  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 4], 1, .5f),
-  similarity-jaccard-prefix(3, [1, 2, 3], 3, [1, 2, 4], 1, .6f),
-
-
-  similarity-jaccard-prefix(3, [1, 2, 3], 9, [1, 2, 3], 1, .5f),
-
-  similarity-jaccard-prefix(4, [1, 2, 3, 4], 2, [1, 2], 1, .5f),
-  similarity-jaccard-prefix(4, [1, 2, 3, 4], 4, [1, 2], 1, .33f)
-]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints.aql
new file mode 100644
index 0000000..b48bdd3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints.aql
@@ -0,0 +1,23 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted-check_ints.adm";
+
+let $a := [ ]
+let $b := [1, 2, 3, 4, 5]
+let $c := [1, 2, 3, 4, 5, 8, 9]
+let $d := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
+let $results :=
+[
+  similarity-jaccard-sorted-check($a, $b, 0.0f),
+  similarity-jaccard-sorted-check($b, $a, 0.0f),
+  similarity-jaccard-sorted-check($a, $b, 0.1f),
+  similarity-jaccard-sorted-check($b, $a, 0.1f),
+  similarity-jaccard-sorted-check($c, $d, 0.6f),
+  similarity-jaccard-sorted-check($d, $c, 0.6f),
+  similarity-jaccard-sorted-check($c, $d, 0.8f),
+  similarity-jaccard-sorted-check($d, $c, 0.8f)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query.aql
new file mode 100644
index 0000000..aa5b067
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted-check_query.adm";
+
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Cooperative Transactions for Environments")
+let $jacc := similarity-jaccard-sorted-check($paper_tokens, $query_tokens, 0.5f)
+where $jacc[0]
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings.aql
new file mode 100644
index 0000000..3fdf844
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings.aql
@@ -0,0 +1,29 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted-check_strings.adm";
+
+let $a := [ ]
+let $b := ["abc", "bcd", "cde", "def", "efg"]
+let $c := ["abc", "bcd", "cde", "def", "efg", "hij", "ijk"]
+let $d := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $e := ["Abc", "bCd", "cdE", "DEf", "eFG", "HiJ", "IJK"]
+let $f := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $results :=
+[
+  similarity-jaccard-sorted-check($a, $b, 0.0f),
+  similarity-jaccard-sorted-check($b, $a, 0.0f),
+  similarity-jaccard-sorted-check($a, $b, 0.1f),
+  similarity-jaccard-sorted-check($b, $a, 0.1f),
+  similarity-jaccard-sorted-check($c, $d, 0.6f),
+  similarity-jaccard-sorted-check($d, $c, 0.6f),
+  similarity-jaccard-sorted-check($c, $d, 0.8f),
+  similarity-jaccard-sorted-check($d, $c, 0.8f),
+  similarity-jaccard-sorted-check($e, $f, 0.6f),
+  similarity-jaccard-sorted-check($f, $e, 0.6f),
+  similarity-jaccard-sorted-check($e, $f, 0.8f),
+  similarity-jaccard-sorted-check($f, $e, 0.8f)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints.aql
new file mode 100644
index 0000000..5fefbf5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints.aql
@@ -0,0 +1,19 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted_ints.adm";
+
+let $a := [ ]
+let $b := [1, 2, 3, 4, 5]
+let $c := [1, 2, 3, 4, 5, 8, 9]
+let $d := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
+let $results := 
+[
+  similarity-jaccard-sorted($a, $b),
+  similarity-jaccard-sorted($b, $a),
+  similarity-jaccard-sorted($c, $d),
+  similarity-jaccard-sorted($d, $c)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query.aql
new file mode 100644
index 0000000..a2373af
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query.aql
@@ -0,0 +1,28 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted_query.adm";
+
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Cooperative Transactions for Environments")
+where similarity-jaccard-sorted($paper_tokens, $query_tokens) >= 0.5
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings.aql
new file mode 100644
index 0000000..67a87d1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings.aql
@@ -0,0 +1,23 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard-sorted_strings.adm";
+
+let $a := [ ]
+let $b := ["abc", "bcd", "cde", "def", "efg"]
+let $c := ["abc", "bcd", "cde", "def", "efg", "hij", "ijk"]
+let $d := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $e := ["Abc", "bCd", "cdE", "DEf", "eFG", "HiJ", "IJK"]
+let $f := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $results :=
+[
+  similarity-jaccard-sorted($a, $b),
+  similarity-jaccard-sorted($b, $a),
+  similarity-jaccard-sorted($c, $d),
+  similarity-jaccard-sorted($d, $c),
+  similarity-jaccard-sorted($e, $f),
+  similarity-jaccard-sorted($f, $e)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_01.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_01.aql
deleted file mode 100644
index ae0747e..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_01.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard_01.adm";
-
-let $a := [1, 2, 3, 4, 5, 8, 9]
-let $b := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
-let $jacc := similarity-jaccard($a, $b)
-return $jacc
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_02.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_02.aql
deleted file mode 100644
index d0a7e1f..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_02.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard_02.adm";
-
-let $a := ["a"]
-let $b := ["b"]
-let $jacc := similarity-jaccard($a, $b)
-return $jacc
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_03.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_03.aql
deleted file mode 100644
index fd5dd21..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_03.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to nc1:"rttest/similarity_similarity-jaccard_03.adm";
-
-let $a := ["a"]
-let $b := [ ]
-let $jacc := similarity-jaccard($a, $b)
-return $jacc
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints.aql
new file mode 100644
index 0000000..ee20d00
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints.aql
@@ -0,0 +1,23 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard_ints.adm";
+
+let $a := [ ]
+let $b := [1, 2, 3, 4, 5]
+let $c := [1, 2, 3, 4, 5, 8, 9]
+let $d := [1, 2, 3, 4, 5, 7, 8, 9, 10, 11]
+let $e := [4, 3, 5, 8, 9, 2, 1]
+let $f := [7, 5, 8, 9, 3, 10, 1, 2, 11, 4]
+let $results :=
+[
+  similarity-jaccard($a, $b),
+  similarity-jaccard($b, $a),
+  similarity-jaccard($c, $d),
+  similarity-jaccard($d, $c),
+  similarity-jaccard($e, $f),
+  similarity-jaccard($f, $e)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query.aql
new file mode 100644
index 0000000..05f3a61
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query.aql
@@ -0,0 +1,28 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  partitioned by key id on group1;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
+write output to nc1:"rttest/similarity_similarity-jaccard_query.adm";
+
+for $paper in dataset('DBLP')
+let $paper_tokens := word-tokens($paper.title)
+let $query_tokens := word-tokens("Transactions for Cooperative Environments")
+where similarity-jaccard($paper_tokens, $query_tokens) >= 0.5f
+return $paper
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings.aql
new file mode 100644
index 0000000..107d92f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings.aql
@@ -0,0 +1,27 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/similarity_similarity-jaccard_strings.adm";
+
+let $a := [ ]
+let $b := ["abc", "bcd", "cde", "def", "efg"]
+let $c := ["abc", "bcd", "cde", "def", "efg", "hij", "ijk"]
+let $d := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $e := ["efg", "abc", "cde", "def", "hij", "ijk", "bcd"]
+let $f := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $g := ["Efg", "aBc", "cdE", "DEf", "hIJ", "IjK", "BCD"]
+let $h := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $results :=
+[
+  similarity-jaccard($a, $b),
+  similarity-jaccard($b, $a),
+  similarity-jaccard($c, $d),
+  similarity-jaccard($d, $c),
+  similarity-jaccard($e, $f),
+  similarity-jaccard($f, $e),
+  similarity-jaccard($g, $h),
+  similarity-jaccard($h, $g)
+]
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string1.aql b/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string1.aql
new file mode 100644
index 0000000..a97e441
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string1.aql
@@ -0,0 +1,9 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_codepoint-to-string1.adm";
+
+let $x :=  [20013, 25991, 23383, 31526]
+let $c := codepoint-to-string($x)
+return {"result1": $c}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string2.aql b/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string2.aql
new file mode 100644
index 0000000..829a082
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/codepoint-to-string2.aql
@@ -0,0 +1,12 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_codepoint-to-string2.adm";
+
+let $x1 :=  []
+let $c1 := codepoint-to-string($x1)
+
+let $x2 :=  [97,98,99]
+let $c2 := codepoint-to-string($x2)
+return {"f1": $c1, "f2" : $c2}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/concat1.aql b/asterix-app/src/test/resources/runtimets/queries/string/concat1.aql
new file mode 100644
index 0000000..a6ce63b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/concat1.aql
@@ -0,0 +1,12 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_concat1.adm";
+
+let $x :=  ["aa", "25991", "bb", "31526"]
+let $c := string-concat($x)
+
+let $x1 :=  []
+let $c1 := string-concat($x1)
+return {"result1": $c,"result2": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with1.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with1.aql
new file mode 100644
index 0000000..98e0624
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/end-with1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_end-with1.adm";
+
+let $c1 := end-with("hello world","werld")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with2.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with2.aql
new file mode 100644
index 0000000..0e53f94
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/end-with2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_end-with2.adm";
+
+let $c1 := end-with("hello world"," world")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with3.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with3.aql
new file mode 100644
index 0000000..f71c320
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/end-with3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_end-with3.adm";
+
+let $c1 := end-with("ends","")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with4.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with4.aql
new file mode 100644
index 0000000..ca02d57
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/end-with4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_end-with4.adm";
+
+let $c1 := end-with("ends","ss")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with5.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with5.aql
new file mode 100644
index 0000000..510beb3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/end-with5.aql
@@ -0,0 +1,14 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_end-with5.adm";
+
+let $c1 := end-with("ends","s")
+let $c2 := end-with("start",null)
+let $c3 := end-with(null,null)
+let $c4 := end-with("",null)
+let $c5 := end-with("","")
+let $c6 := end-with(null,"")
+
+return {"f1": $c1, "f2": $c2, "f3": $c3, "f4": $c4, "f5": $c5, "f6": $c6}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/length.aql b/asterix-app/src/test/resources/runtimets/queries/string/length.aql
new file mode 100644
index 0000000..d78d986
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/length.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_length.adm";
+
+let $c1 := string-length("hellow")
+let $c2 := string-length("")
+let $c3 := string-length(null)
+return {"result1": $c1, "result2": $c2, "result3": $c3}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/lowercase.aql b/asterix-app/src/test/resources/runtimets/queries/string/lowercase.aql
new file mode 100644
index 0000000..4c45007
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/lowercase.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_lowercase.adm";
+
+let $c1 := lowercase("HEllow")
+let $c2 := lowercase("")
+let $c3 := lowercase(null)
+return {"result1": $c1, "result2": $c2, "result3": $c3}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches1.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches1.aql
new file mode 100644
index 0000000..f81932a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches1.adm";
+
+let $c1 := matches("abracadabra","bra")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches2.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches2.aql
new file mode 100644
index 0000000..af5246e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches2.adm";
+
+let $c1 := matches("abracadabra","^a.*a$")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches21.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches21.aql
new file mode 100644
index 0000000..19f9591
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches21.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches21.adm";
+
+let $c1 := matches2("abracadabra","Bra","")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches22.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches22.aql
new file mode 100644
index 0000000..b8d7b96
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches22.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches22.adm";
+
+let $c1 := matches2("abracadabra","Bra","i")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches23.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches23.aql
new file mode 100644
index 0000000..508699d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches23.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches23.adm";
+
+let $c1 := matches2("helloworld","hello world","x")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches3.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches3.aql
new file mode 100644
index 0000000..19c6350
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matches3.adm";
+
+let $c1 := matches("abracadabra","^bra")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matchesnull.aql b/asterix-app/src/test/resources/runtimets/queries/string/matchesnull.aql
new file mode 100644
index 0000000..8e2f1a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matchesnull.aql
@@ -0,0 +1,13 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_matchesnull.adm";
+
+let $c1 := matches("helloworld",null)
+let $c2 := matches("",null)
+let $c3 := matches(null,null)
+let $c4 := matches2("helloworld",null, "")
+let $c5 := matches2("",null, "i")
+let $c6 := matches2(null,null, null)
+return {"result1": $c1, "result2": $c2, "result3": $c3, "result4": $c4, "result5": $c5, "result6": $c6}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace1.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace1.aql
new file mode 100644
index 0000000..3a9e932
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace1.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_replace1.adm";
+
+let $c1 := replace("abracadabra", "a", "")
+let $c2 := replace("abracadabra", "a(.)", "a$1$1")
+let $c3 := replace("darted", "^(.*?)d(.*)$", "$1c$2")
+return {"result1": $c1, "result2": $c2, "result3": $c3}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace2.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace2.aql
new file mode 100644
index 0000000..9b924f2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_replace2.adm";
+
+let $c1 := replace("abracadabra", "bra", "*")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace21.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace21.aql
new file mode 100644
index 0000000..77c0c90
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace21.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_replace21.adm";
+
+let $c1 := replace2("abracadabra","Bra", "kkk" , "")
+let $c2 := replace2("abracadabra","Bra", "kkk" ,"i")
+let $c3 := replace2("helloworld","hello world", "kkk" , "x")
+return {"result1": $c1,"result2": $c2,"result3": $c3}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace22.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace22.aql
new file mode 100644
index 0000000..2a0d5d6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace22.aql
@@ -0,0 +1,15 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_replace22.adm";
+
+let $c1 := replace2("abracadabra","", null , null)
+let $c2 := replace2("abracadabra","bra", "XXX" ,"")
+let $c3 := replace2(null,"hello world", "XxXx" , "x")
+let $c4 := replace2("abracadabra","bra", "XXX" ,null)
+let $c5 := replace2("abracadabra",null, "XXX" ,null)
+let $c6 := replace2("abracadabra","Bra", null ,"i")
+let $c7 := replace2("abracadabra","Bra", "" ,"i")
+let $c8 := replace2("abracadabra","", "XXX" ,"")
+return {"result1": $c1,"result2": $c2,"result3": $c3,"result4": $c4,"result5": $c5,"result6": $c6,"result7": $c7,"result8": $c8}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace3.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace3.aql
new file mode 100644
index 0000000..c1892dd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_replace3.adm";
+
+let $c1 := replace("abracadabra", "a.*a", "*")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/start-with1.aql b/asterix-app/src/test/resources/runtimets/queries/string/start-with1.aql
new file mode 100644
index 0000000..11ac150
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/start-with1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_start-with1.adm";
+
+let $c1 := start-with("start","st")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/start-with2.aql b/asterix-app/src/test/resources/runtimets/queries/string/start-with2.aql
new file mode 100644
index 0000000..6d02c9a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/start-with2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_start-with2.adm";
+
+let $c1 := start-with("start","t")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/start-with3.aql b/asterix-app/src/test/resources/runtimets/queries/string/start-with3.aql
new file mode 100644
index 0000000..a70dc33
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/start-with3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_start-with3.adm";
+
+let $c1 := start-with("start","start")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/start-with4.aql b/asterix-app/src/test/resources/runtimets/queries/string/start-with4.aql
new file mode 100644
index 0000000..3629093
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/start-with4.aql
@@ -0,0 +1,14 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_start-with4.adm";
+
+let $c1 := start-with("start","")
+let $c2 := start-with("start",null)
+let $c3 := start-with(null,null)
+let $c4 := start-with("",null)
+let $c5 := start-with("","")
+let $c6 := start-with(null,"")
+
+return {"f1": $c1, "f2": $c2, "f3": $c3, "f4": $c4, "f5": $c5, "f6": $c6}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/start-with5.aql b/asterix-app/src/test/resources/runtimets/queries/string/start-with5.aql
new file mode 100644
index 0000000..33a0b8e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/start-with5.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_start-with5.adm";
+
+let $c1 := start-with("","s")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-concat1.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-concat1.aql
new file mode 100644
index 0000000..b31e33c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-concat1.aql
@@ -0,0 +1,9 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-concat1.adm";
+
+let $x :=  ["aa", "25991", "bb", "31526"]
+let $c := string-concat($x)
+return {"result1": $c}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal1.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal1.aql
new file mode 100644
index 0000000..62ec56d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-equal1.adm";
+
+let $c1 := string-equal("test","tess")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal2.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal2.aql
new file mode 100644
index 0000000..a102802
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-equal2.adm";
+
+let $c1 := string-equal("test","test")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal3.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal3.aql
new file mode 100644
index 0000000..2749390
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-equal3.adm";
+
+let $c1 := string-equal("test11","test")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal4.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal4.aql
new file mode 100644
index 0000000..f1cf243
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal4.aql
@@ -0,0 +1,11 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-equal4.adm";
+
+let $c1 := string-equal("","")
+let $c3 := string-equal(null,"")
+let $c4 := string-equal("",null)
+let $c5 := string-equal(null,null)
+return {"result1": $c1, "result3": $c3, "result4": $c4, "result5": $c5}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-join1.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-join1.aql
new file mode 100644
index 0000000..0ea2bdd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-join1.aql
@@ -0,0 +1,11 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-join1.adm";
+
+let $x :=  ["aa", "25991", "bb", "31526"]
+let $s := "::"
+let $c := string-join($x,$s)
+let $c1 := string-join($x,"")
+return {"result0": $c,"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint.aql
new file mode 100644
index 0000000..2f1daba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint.aql
@@ -0,0 +1,9 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-to-codepoint.adm";
+
+let $x :=  "abcd"
+let $c := string-to-codepoint($x)
+return {"result1": $c}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint1.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint1.aql
new file mode 100644
index 0000000..5bcfd16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint1.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_string-to-codepoint1.adm";
+
+let $x :=  ""
+let $c := string-to-codepoint($x)
+
+return {"result1": $c}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-after-1.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-1.aql
new file mode 100644
index 0000000..6c45bc8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-after-1.adm";
+
+let $c1 := substring-after("HEllow","El")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-after-2.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-2.aql
new file mode 100644
index 0000000..78d6cad
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-after-2.adm";
+
+let $c1 := substring-after("HEllow","1")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-after-3.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-3.aql
new file mode 100644
index 0000000..91c3cab
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-after-3.adm";
+
+let $c1 := substring-after("HEllow","HEllow")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-after-4.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-4.aql
new file mode 100644
index 0000000..73a983f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-after-4.aql
@@ -0,0 +1,12 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-after-4.adm";
+
+let $c1 := substring-after("HEllow","")
+let $c2 := substring-after("HEllow",null)
+let $c3 := substring-after("",null)
+let $c4 := substring-after("","")
+let $c5 := substring-after(null,null)
+return {"result1": $c1, "result2": $c2, "result3": $c3, "result4": $c4, "result5": $c5}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-before-1.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-1.aql
new file mode 100644
index 0000000..1fe3a36
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-before-1.adm";
+
+let $c1 := substring-before("HEllow","ll")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-before-2.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-2.aql
new file mode 100644
index 0000000..4f59c43
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-before-2.adm";
+
+let $c1 := substring-before("HEllow","HEllow")
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring-before-3.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-3.aql
new file mode 100644
index 0000000..9e40615
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring-before-3.aql
@@ -0,0 +1,12 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring-before-3.adm";
+
+let $c1 := substring-before("HEllow","")
+let $c2 := substring-before("HEllow",null)
+let $c3 := substring-before("",null)
+let $c4 := substring-before("","")
+let $c5 := substring-before(null,null)
+return {"result1": $c1, "result2": $c2, "result3": $c3, "result4": $c4, "result5": $c5}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-1.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-1.aql
new file mode 100644
index 0000000..78beb56
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-1.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring2-1.adm";
+
+let $c1 := substring2("HEllow",2)
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-2.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-2.aql
new file mode 100644
index 0000000..b310c9f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-2.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring2-2.adm";
+
+let $c1 := substring2("HEllow",0)
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-3.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-3.aql
new file mode 100644
index 0000000..70725b6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-3.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring2-3.adm";
+
+let $c1 := substring2("HEllow",10)
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-4.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-4.aql
new file mode 100644
index 0000000..1a1065a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-4.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/string_substring2-4.adm";
+
+let $c1 := substring2("HEllow",-1)
+return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/results/constructor/add-null.adm b/asterix-app/src/test/resources/runtimets/results/constructor/add-null.adm
new file mode 100644
index 0000000..19765bd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/constructor/add-null.adm
@@ -0,0 +1 @@
+null
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/drop-index.adm b/asterix-app/src/test/resources/runtimets/results/dml/drop-index.adm
new file mode 100644
index 0000000..f3c688b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/drop-index.adm
@@ -0,0 +1 @@
+{ "unique1": 84, "unique2": 10, "two": 0, "four": 0, "ten": 4, "twenty": 4, "onePercent": 84, "tenPercent": 4, "twentyPercent": 4, "fiftyPercent": 0, "unique3": 84, "evenOnePercent": 168, "oddOnePercent": 169, "stringu1": "DGAAAAXXXXXXXXXXXXXXXXXXX", "stringu2": "KAAAAAXXXXXXXXXXXXXXXXXXX", "string4": "OOOOXXXXXXXXXXXXXXXXXXXXXX" }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.1.adm
index e3d63ca..63b0d8c 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.1.adm
@@ -1,3 +1,3 @@
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": < "biking", "singing" > }, "sim": 1.0f }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": < "biking", "acting", "painting" > }, "sim": 0.5f }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "sim": 0.5f }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} }, "sim": 1.0f }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": {{ "biking", "acting", "painting" }} }, "sim": 0.5f }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "sim": 0.5f }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.adm
index c1c7413..814df80 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_1.adm
@@ -1,5 +1,5 @@
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": < "biking", "acting", "painting" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "user2": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "user2": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": < "kayaking", "running" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": < "biking", "singing" > } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": {{ "biking", "acting", "painting" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "user2": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "user2": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": {{ "kayaking", "running" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_2.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_2.adm
index c1c7413..814df80 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_2.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-lot-aqlplus_2.adm
@@ -1,5 +1,5 @@
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": < "biking", "acting", "painting" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "user2": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "user2": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": < "kayaking", "running" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": < "biking", "singing" > } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": {{ "biking", "acting", "painting" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "user2": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "user2": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "user2": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": {{ "kayaking", "running" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-aqlplus_3.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-aqlplus_3.adm
index e96c143..693874d 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-aqlplus_3.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-aqlplus_3.adm
@@ -1,15 +1,15 @@
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": < "running", "swimming", "biking" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": < "hiking", "running", "swimming", "biking" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1060, "name": "Mckenzie Neitzke", "lottery_numbers": [  ], "interests": < "hiking", "biking", "swimming" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": < "swimming", "hiking", "running" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": < "singing", "acting" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": < "acting", "singing" > } }
-{ "user": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": < "biking", "acting", "painting" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
-{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": < "running", "swimming", "biking" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": < "hiking", "running", "swimming", "biking" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1060, "name": "Mckenzie Neitzke", "lottery_numbers": [  ], "interests": < "hiking", "biking", "swimming" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": < "swimming", "hiking", "running" > } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": {{ "running", "swimming", "biking" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1060, "name": "Mckenzie Neitzke", "lottery_numbers": [  ], "interests": {{ "hiking", "biking", "swimming" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": {{ "swimming", "hiking", "running" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": {{ "singing", "acting" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": {{ "acting", "singing" }} } }
+{ "user": { "uid": 50, "name": "Lance Pracht", "lottery_numbers": [ 10, 20, 41 ], "interests": {{ "biking", "acting", "painting" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
+{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": {{ "running", "swimming", "biking" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1060, "name": "Mckenzie Neitzke", "lottery_numbers": [  ], "interests": {{ "hiking", "biking", "swimming" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": {{ "swimming", "hiking", "running" }} } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.adm
index 885a976..68524fa 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-int-vis-user-lot-aqlplus_1.adm
@@ -5,4 +5,4 @@
 { "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": {{ "acting", "singing" }} }, "user2": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} } }
 { "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": {{ "acting", "singing" }} }, "user2": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} } }
 { "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} }, "user2": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} }, "user2": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} } }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-aqlplus_3.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-aqlplus_3.adm
index 6ddbfd1..2b94840 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-aqlplus_3.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-aqlplus_3.adm
@@ -1,18 +1,18 @@
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": < "hiking", "running", "swimming" > }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": < "singing", "acting" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": < "running", "swimming", "biking" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": < "hiking" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": < "acting", "singing" > } }
-{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": < "biking", "painting" > }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": < "swimming", "hiking", "running" > } }
-{ "user": { "uid": 30, "name": "Marvella Loud", "lottery_numbers": [ 40, 41, 42 ], "interests": <  > }, "visitor": { "vid": 1040, "name": "Luella Schweinert", "lottery_numbers": [ 41, 42, 43 ], "interests": < "hiking", "kayaking" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": < "hiking", "running", "swimming", "biking" > } }
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
-{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": < "hiking" > } }
-{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": < "acting", "singing" > } }
-{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": < "acting", "running" > }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": < "swimming", "hiking", "running" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": < "acting", "running" > } }
-{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": < "running", "hiking", "biking" > }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": < "singing", "acting" > } }
-{ "user": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": < "biking", "singing" > }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": < "hiking", "running", "swimming", "biking" > } }
-{ "user": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": < "biking", "singing" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
-{ "user": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": < "kayaking", "running" > }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": < "running", "swimming", "biking" > } }
-{ "user": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": < "kayaking", "running" > }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": < "hiking" > } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 10, "name": "Jodi Rotruck", "lottery_numbers": [ 10, 15, 20 ], "interests": {{ "hiking", "running", "swimming" }} }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": {{ "singing", "acting" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": {{ "running", "swimming", "biking" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "hiking" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": {{ "acting", "singing" }} } }
+{ "user": { "uid": 20, "name": "Clint Coil", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "biking", "painting" }} }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": {{ "swimming", "hiking", "running" }} } }
+{ "user": { "uid": 30, "name": "Marvella Loud", "lottery_numbers": [ 40, 41, 42 ], "interests": {{  }} }, "visitor": { "vid": 1040, "name": "Luella Schweinert", "lottery_numbers": [ 41, 42, 43 ], "interests": {{ "hiking", "kayaking" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
+{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "hiking" }} } }
+{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} }, "visitor": { "vid": 1090, "name": "Jackson Fillerup", "lottery_numbers": [ 25, 30, 35 ], "interests": {{ "acting", "singing" }} } }
+{ "user": { "uid": 60, "name": "Larry Gothier", "lottery_numbers": [ 25, 30, 31 ], "interests": {{ "acting", "running" }} }, "visitor": { "vid": 1100, "name": "Laree Savasta", "lottery_numbers": [ 20, 30, 31 ], "interests": {{ "swimming", "hiking", "running" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1010, "name": "Alex Ascher", "lottery_numbers": [ 10, 15, 30 ], "interests": {{ "acting", "running" }} } }
+{ "user": { "uid": 70, "name": "Obdulia Dicosmo", "lottery_numbers": [ 10, 15, 40 ], "interests": {{ "running", "hiking", "biking" }} }, "visitor": { "vid": 1050, "name": "Harold Pendelton", "lottery_numbers": [ 10, 12, 15 ], "interests": {{ "singing", "acting" }} } }
+{ "user": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} }, "visitor": { "vid": 1030, "name": "Shanna Cuaresma", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "hiking", "running", "swimming", "biking" }} } }
+{ "user": { "uid": 80, "name": "Elias Leonardo", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "biking", "singing" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
+{ "user": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": {{ "kayaking", "running" }} }, "visitor": { "vid": 1020, "name": "Hank Friley", "lottery_numbers": [ 20, 25 ], "interests": {{ "running", "swimming", "biking" }} } }
+{ "user": { "uid": 90, "name": "Myrtice Cubias", "lottery_numbers": [ 20, 25, 43 ], "interests": {{ "kayaking", "running" }} }, "visitor": { "vid": 1080, "name": "Mohammad Ohr", "lottery_numbers": [ 20, 25, 30 ], "interests": {{ "hiking" }} } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-int-aqlplus_2.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-int-aqlplus_2.adm
index e70ef1d..d2f4951 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-int-aqlplus_2.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/user-vis-lot-int-aqlplus_2.adm
@@ -1 +1 @@
-{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": < "singing", "acting", "running" > }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": < "acting", "running", "painting" > } }
+{ "user": { "uid": 40, "name": "Tamie Pollara", "lottery_numbers": [ 45, 46, 47 ], "interests": {{ "singing", "acting", "running" }} }, "visitor": { "vid": 1070, "name": "Crystal Isabella", "lottery_numbers": [ 41, 45, 46 ], "interests": {{ "acting", "running", "painting" }} } }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-edit-distance.adm
new file mode 100644
index 0000000..0f9b451
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-edit-distance.adm
@@ -0,0 +1,13 @@
+{ "arec": { "cid": 8, "name": "Audria Haylett", "age": 44, "address": { "number": 4872, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cooking", "Fishing", "Video Games" ], "children": [ { "name": "Lacie Haylett", "age": 19 } ] }, "brec": { "cid": 311, "name": "Ria Haflett", "age": 14, "address": { "number": 9513, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Jimmie Haflett", "age": null }, { "name": "Dario Haflett", "age": null }, { "name": "Robbyn Haflett", "age": null } ] } }
+{ "arec": { "cid": 102, "name": "Melany Rotan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Christiana Rotan", "age": 21 }, { "name": "Lavina Rotan", "age": null }, { "name": "Billy Rotan", "age": null } ] }, "brec": { "cid": 378, "name": "Melany Matias", "age": 10, "address": { "number": 8838, "street": "Main St.", "city": "Seattle" }, "interests": [ "Coffee", "Tennis", "Bass" ], "children": [ { "name": "Earnestine Matias", "age": null }, { "name": "Lore Matias", "age": null } ] } }
+{ "arec": { "cid": 104, "name": "Neda Dilts", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Nona Dilts", "age": 28 }, { "name": "Wm Dilts", "age": null }, { "name": "Svetlana Dilts", "age": 46 }, { "name": "Iva Dilts", "age": 59 } ] }, "brec": { "cid": 569, "name": "Beata Diles", "age": 88, "address": { "number": 2198, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Myrtice Diles", "age": 46 }, { "name": "Stella Diles", "age": null }, { "name": "Rowena Diles", "age": 26 } ] } }
+{ "arec": { "cid": 135, "name": "Josette Dries", "age": null, "address": null, "interests": [ "Base Jumping", "Movies" ], "children": [ { "name": "Ben Dries", "age": 36 }, { "name": "Wm Dries", "age": 29 } ] }, "brec": { "cid": 855, "name": "Rosette Reen", "age": 57, "address": { "number": 2767, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Basketball" ], "children": [  ] } }
+{ "arec": { "cid": 204, "name": "Londa Herdt", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marnie Herdt", "age": 47 } ] }, "brec": { "cid": 247, "name": "Minda Heron", "age": 25, "address": { "number": 1629, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [  ] } }
+{ "arec": { "cid": 205, "name": "Moises Plake", "age": null, "address": null, "interests": [ "Puzzles", "Computers" ], "children": [  ] }, "brec": { "cid": 401, "name": "Moises Jago", "age": 27, "address": { "number": 3773, "street": "Main St.", "city": "San Jose" }, "interests": [ "Music" ], "children": [ { "name": "Shoshana Jago", "age": null }, { "name": "Juliet Jago", "age": null }, { "name": "Berneice Jago", "age": 13 } ] } }
+{ "arec": { "cid": 209, "name": "Donnette Kreb", "age": null, "address": null, "interests": [ "Puzzles", "Cooking", "Tennis", "Tennis" ], "children": [ { "name": "Hobert Kreb", "age": null }, { "name": "Ray Kreb", "age": null }, { "name": "Carmel Kreb", "age": 56 }, { "name": "Lise Kreb", "age": null } ] }, "brec": { "cid": 829, "name": "Donnette Lebel", "age": null, "address": null, "interests": [ "Tennis", "Coffee", "Running", "Fishing" ], "children": [ { "name": "Junior Lebel", "age": null } ] } }
+{ "arec": { "cid": 272, "name": "Frederick Valla", "age": 15, "address": { "number": 6805, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Carroll Valla", "age": null } ] }, "brec": { "cid": 797, "name": "Frederica Kale", "age": 77, "address": { "number": 6861, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Bass" ], "children": [ { "name": "Shanice Kale", "age": null }, { "name": "Soraya Kale", "age": 64 }, { "name": "Laurena Kale", "age": 57 } ] } }
+{ "arec": { "cid": 464, "name": "Petra Kinsel", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Janise Kinsel", "age": null }, { "name": "Donnie Kinsel", "age": 26 }, { "name": "Joana Kinsel", "age": 12 } ] }, "brec": { "cid": 748, "name": "Petra Ganes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Perry Ganes", "age": null }, { "name": "Krista Ganes", "age": 54 }, { "name": "Kayce Ganes", "age": 52 }, { "name": "Eleni Ganes", "age": null } ] } }
+{ "arec": { "cid": 470, "name": "Yesenia Doyon", "age": 78, "address": { "number": 3641, "street": "7th St.", "city": "Seattle" }, "interests": [ "Databases", "Puzzles" ], "children": [ { "name": "Halley Doyon", "age": null }, { "name": "Teisha Doyon", "age": 33 }, { "name": "Warren Doyon", "age": null } ] }, "brec": { "cid": 997, "name": "Yesenia Gao", "age": 38, "address": { "number": 5990, "street": "View St.", "city": "Portland" }, "interests": [ "Computers", "Computers", "Puzzles", "Puzzles" ], "children": [ { "name": "Jared Gao", "age": 11 }, { "name": "Sang Gao", "age": null }, { "name": "Jeanne Gao", "age": 13 }, { "name": "Lavona Gao", "age": 23 } ] } }
+{ "arec": { "cid": 486, "name": "Willa Patman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ross Patman", "age": 42 }, { "name": "Erin Patman", "age": null }, { "name": "Vannessa Patman", "age": 11 }, { "name": "Hilaria Patman", "age": 28 } ] }, "brec": { "cid": 765, "name": "Mila Barman", "age": null, "address": null, "interests": [ "Coffee", "Puzzles", "Bass", "Wine" ], "children": [ { "name": "Lucienne Barman", "age": null }, { "name": "Marina Barman", "age": null } ] } }
+{ "arec": { "cid": 531, "name": "Camelia Yoes", "age": null, "address": null, "interests": [  ], "children": [  ] }, "brec": { "cid": 574, "name": "Camellia Toxey", "age": 52, "address": { "number": 5437, "street": "Hill St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Deandrea Toxey", "age": null }, { "name": "Danille Toxey", "age": null } ] } }
+{ "arec": { "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] }, "brec": { "cid": 954, "name": "Yolonda Pu", "age": null, "address": null, "interests": [ "Video Games", "Music", "Cooking", "Skiing" ], "children": [ { "name": "Josephina Pu", "age": 35 } ] } }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-jaccard.adm
new file mode 100644
index 0000000..c54133f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ngram-jaccard.adm
@@ -0,0 +1,7 @@
+{ "arec": "Transaction Management in Multidatabase Systems.", "brec": "Overview of Multidatabase Transaction Management" }
+{ "arec": "Transaction Management in Multidatabase Systems.", "brec": "Overview of Multidatabase Transaction Management" }
+{ "arec": "Active Database Systems.", "brec": "Active Database Systems" }
+{ "arec": "Specification and Execution of Transactional Workflows.", "brec": "Specification and Execution of Transactional Workflows" }
+{ "arec": "Integrated Office Systems.", "brec": "Integrated Office Systems" }
+{ "arec": "Integrated Office Systems.", "brec": "Integrated Office Systems" }
+{ "arec": "A Shared View of Sharing  The Treaty of Orlando.", "brec": "A Shared View of Sharing  The Treaty of Orlando" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-edit-distance.adm
new file mode 100644
index 0000000..99d6623
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-edit-distance.adm
@@ -0,0 +1,157 @@
+{ "arec": { "cid": 8, "name": "Audria Haylett", "age": 44, "address": { "number": 4872, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cooking", "Fishing", "Video Games" ], "children": [ { "name": "Lacie Haylett", "age": 19 } ] }, "brec": { "cid": 563, "name": "Deirdre Landero", "age": null, "address": null, "interests": [ "Books", "Fishing", "Video Games" ], "children": [ { "name": "Norman Landero", "age": 59 }, { "name": "Jennine Landero", "age": 45 }, { "name": "Rutha Landero", "age": 19 }, { "name": "Jackie Landero", "age": 29 } ] } }
+{ "arec": { "cid": 16, "name": "Felisa Auletta", "age": 55, "address": { "number": 7737, "street": "View St.", "city": "San Jose" }, "interests": [ "Skiing", "Coffee", "Wine" ], "children": [ { "name": "Rosalia Auletta", "age": 36 } ] }, "brec": { "cid": 273, "name": "Corrinne Seaquist", "age": 24, "address": { "number": 6712, "street": "7th St.", "city": "Portland" }, "interests": [ "Puzzles", "Coffee", "Wine" ], "children": [ { "name": "Mignon Seaquist", "age": null }, { "name": "Leo Seaquist", "age": null } ] } }
+{ "arec": { "cid": 16, "name": "Felisa Auletta", "age": 55, "address": { "number": 7737, "street": "View St.", "city": "San Jose" }, "interests": [ "Skiing", "Coffee", "Wine" ], "children": [ { "name": "Rosalia Auletta", "age": 36 } ] }, "brec": { "cid": 618, "name": "Janella Hurtt", "age": null, "address": null, "interests": [ "Skiing", "Coffee", "Skiing" ], "children": [ { "name": "Lupe Hurtt", "age": 17 }, { "name": "Jae Hurtt", "age": 14 }, { "name": "Evan Hurtt", "age": 45 } ] } }
+{ "arec": { "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }, "brec": { "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] } }
+{ "arec": { "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }, "brec": { "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] } }
+{ "arec": { "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }, "brec": { "cid": 666, "name": "Pamila Burzlaff", "age": 68, "address": { "number": 6543, "street": "View St.", "city": "Portland" }, "interests": [ "Squash", "Cigars", "Movies" ], "children": [  ] } }
+{ "arec": { "cid": 18, "name": "Dewayne Ardan", "age": 32, "address": { "number": 8229, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Wine", "Walking", "Bass" ], "children": [ { "name": "Wen Ardan", "age": null }, { "name": "Sachiko Ardan", "age": 11 }, { "name": "Francis Ardan", "age": 20 } ] }, "brec": { "cid": 846, "name": "Kieth Norlund", "age": 15, "address": { "number": 4039, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Puzzles" ], "children": [ { "name": "Shawn Norlund", "age": null } ] } }
+{ "arec": { "cid": 35, "name": "Saundra Aparo", "age": 86, "address": { "number": 9550, "street": "Lake St.", "city": "Portland" }, "interests": [ "Cigars", "Skiing", "Video Games", "Books" ], "children": [  ] }, "brec": { "cid": 926, "name": "Krishna Barkdull", "age": 31, "address": { "number": 2640, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Skiing", "Video Games", "Coffee" ], "children": [ { "name": "Nilsa Barkdull", "age": null }, { "name": "Denver Barkdull", "age": 10 }, { "name": "Jenell Barkdull", "age": 15 } ] } }
+{ "arec": { "cid": 51, "name": "Simonne Cape", "age": null, "address": null, "interests": [ "Bass", "Bass", "Books" ], "children": [ { "name": "Leland Cape", "age": null }, { "name": "Gearldine Cape", "age": null } ] }, "brec": { "cid": 232, "name": "Joey Potes", "age": null, "address": null, "interests": [ "Bass", "Bass", "Base Jumping" ], "children": [ { "name": "Bobby Potes", "age": null } ] } }
+{ "arec": { "cid": 51, "name": "Simonne Cape", "age": null, "address": null, "interests": [ "Bass", "Bass", "Books" ], "children": [ { "name": "Leland Cape", "age": null }, { "name": "Gearldine Cape", "age": null } ] }, "brec": { "cid": 412, "name": "Devon Szalai", "age": 26, "address": { "number": 2384, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books", "Books" ], "children": [ { "name": "Yolonda Szalai", "age": null }, { "name": "Denita Szalai", "age": null }, { "name": "Priscila Szalai", "age": 10 }, { "name": "Cassondra Szalai", "age": 12 } ] } }
+{ "arec": { "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }, "brec": { "cid": 229, "name": "Raymundo Meurin", "age": null, "address": null, "interests": [ "Bass", "Basketball", "Databases" ], "children": [ { "name": "Mariela Meurin", "age": null } ] } }
+{ "arec": { "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }, "brec": { "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] } }
+{ "arec": { "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }, "brec": { "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] } }
+{ "arec": { "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }, "brec": { "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] } }
+{ "arec": { "cid": 72, "name": "Clarissa Geraldes", "age": 67, "address": { "number": 8248, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Cigars", "Walking", "Databases", "Video Games" ], "children": [ { "name": "Vina Geraldes", "age": 51 } ] }, "brec": { "cid": 919, "name": "Fairy Wansley", "age": 45, "address": { "number": 9020, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Databases", "Video Games" ], "children": [ { "name": "Marvella Wansley", "age": null }, { "name": "Hisako Wansley", "age": null }, { "name": "Shaunta Wansley", "age": null }, { "name": "Gemma Wansley", "age": 21 } ] } }
+{ "arec": { "cid": 73, "name": "Kelsey Flever", "age": 20, "address": { "number": 3555, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Puzzles", "Video Games" ], "children": [ { "name": "Isis Flever", "age": null }, { "name": "Gonzalo Flever", "age": null } ] }, "brec": { "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] } }
+{ "arec": { "cid": 73, "name": "Kelsey Flever", "age": 20, "address": { "number": 3555, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Puzzles", "Video Games" ], "children": [ { "name": "Isis Flever", "age": null }, { "name": "Gonzalo Flever", "age": null } ] }, "brec": { "cid": 734, "name": "Lera Korn", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Cigars" ], "children": [ { "name": "Criselda Korn", "age": 37 } ] } }
+{ "arec": { "cid": 77, "name": "Chantal Parriera", "age": 78, "address": { "number": 5967, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Squash", "Movies", "Coffee" ], "children": [  ] }, "brec": { "cid": 909, "name": "Mariko Sharar", "age": null, "address": null, "interests": [ "Squash", "Movies", "Computers" ], "children": [  ] } }
+{ "arec": { "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }, "brec": { "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] } }
+{ "arec": { "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }, "brec": { "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] } }
+{ "arec": { "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }, "brec": { "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] } }
+{ "arec": { "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }, "brec": { "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] } }
+{ "arec": { "cid": 84, "name": "Huong Kachel", "age": null, "address": null, "interests": [ "Music", "Tennis", "Base Jumping" ], "children": [ { "name": "Katlyn Kachel", "age": 40 }, { "name": "Sherman Kachel", "age": null }, { "name": "Susana Kachel", "age": 32 } ] }, "brec": { "cid": 326, "name": "Tad Tellers", "age": null, "address": null, "interests": [ "Books", "Tennis", "Base Jumping" ], "children": [ { "name": "Fannie Tellers", "age": null } ] } }
+{ "arec": { "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] }, "brec": { "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] } }
+{ "arec": { "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] }, "brec": { "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] } }
+{ "arec": { "cid": 101, "name": "Meaghan Vandel", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Books" ], "children": [ { "name": "Larissa Vandel", "age": null } ] }, "brec": { "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] } }
+{ "arec": { "cid": 101, "name": "Meaghan Vandel", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Books" ], "children": [ { "name": "Larissa Vandel", "age": null } ] }, "brec": { "cid": 967, "name": "Melida Laliotis", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Coffee", "Books" ], "children": [ { "name": "Lai Laliotis", "age": 52 }, { "name": "Jillian Laliotis", "age": 11 } ] } }
+{ "arec": { "cid": 115, "name": "Jason Oakden", "age": 89, "address": { "number": 8182, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Music", "Basketball", "Movies" ], "children": [ { "name": "Johnson Oakden", "age": null }, { "name": "Neva Oakden", "age": null }, { "name": "Juliann Oakden", "age": null }, { "name": "Elmer Oakden", "age": null } ] }, "brec": { "cid": 827, "name": "Clementina Papin", "age": null, "address": null, "interests": [ "Music", "Basketball", "Cigars" ], "children": [ { "name": "Catina Papin", "age": null }, { "name": "Demetrius Papin", "age": 59 }, { "name": "Marylou Papin", "age": 12 }, { "name": "Apryl Papin", "age": 16 } ] } }
+{ "arec": { "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }, "brec": { "cid": 397, "name": "Blake Kealy", "age": 34, "address": { "number": 2156, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Databases", "Wine", "Cigars" ], "children": [ { "name": "Lorenza Kealy", "age": null }, { "name": "Beula Kealy", "age": 15 }, { "name": "Kristofer Kealy", "age": null }, { "name": "Shayne Kealy", "age": null } ] } }
+{ "arec": { "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }, "brec": { "cid": 636, "name": "Babara Shore", "age": 83, "address": { "number": 9452, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Candy Shore", "age": 58 }, { "name": "Nanci Shore", "age": null }, { "name": "Asia Shore", "age": null } ] } }
+{ "arec": { "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }, "brec": { "cid": 992, "name": "Staci Alexandropoul", "age": null, "address": null, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Casimira Alexandropoul", "age": null }, { "name": "Kena Alexandropoul", "age": 54 }, { "name": "Ellie Alexandropoul", "age": null }, { "name": "Ambrose Alexandropoul", "age": null } ] } }
+{ "arec": { "cid": 137, "name": "Camellia Pressman", "age": 81, "address": { "number": 3947, "street": "Park St.", "city": "Seattle" }, "interests": [ "Movies", "Books", "Bass" ], "children": [ { "name": "Dwana Pressman", "age": null }, { "name": "Johnathan Pressman", "age": null }, { "name": "Kasey Pressman", "age": null }, { "name": "Mitch Pressman", "age": null } ] }, "brec": { "cid": 923, "name": "Bobbi Ursino", "age": null, "address": null, "interests": [ "Movies", "Books", "Walking" ], "children": [ { "name": "Shon Ursino", "age": null }, { "name": "Lorean Ursino", "age": null } ] } }
+{ "arec": { "cid": 139, "name": "Micheline Argenal", "age": null, "address": null, "interests": [ "Bass", "Walking", "Movies" ], "children": [ { "name": "Joye Argenal", "age": 51 }, { "name": "Richard Argenal", "age": 46 }, { "name": "Sarah Argenal", "age": 21 }, { "name": "Jacinda Argenal", "age": 21 } ] }, "brec": { "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] } }
+{ "arec": { "cid": 141, "name": "Adena Klockars", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Bass", "Cigars" ], "children": [  ] }, "brec": { "cid": 794, "name": "Annabel Leins", "age": 75, "address": { "number": 9761, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Bass", "Computers", "Bass", "Cigars" ], "children": [ { "name": "Oswaldo Leins", "age": 21 } ] } }
+{ "arec": { "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }, "brec": { "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] } }
+{ "arec": { "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }, "brec": { "cid": 666, "name": "Pamila Burzlaff", "age": 68, "address": { "number": 6543, "street": "View St.", "city": "Portland" }, "interests": [ "Squash", "Cigars", "Movies" ], "children": [  ] } }
+{ "arec": { "cid": 160, "name": "Yevette Chanez", "age": null, "address": null, "interests": [ "Bass", "Wine", "Coffee" ], "children": [ { "name": "Walter Chanez", "age": 11 }, { "name": "Pa Chanez", "age": 27 } ] }, "brec": { "cid": 299, "name": "Jacob Wainman", "age": 76, "address": { "number": 4551, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Wine", "Coffee" ], "children": [ { "name": "Abram Wainman", "age": 28 }, { "name": "Ramonita Wainman", "age": 18 }, { "name": "Sheryll Wainman", "age": null } ] } }
+{ "arec": { "cid": 160, "name": "Yevette Chanez", "age": null, "address": null, "interests": [ "Bass", "Wine", "Coffee" ], "children": [ { "name": "Walter Chanez", "age": 11 }, { "name": "Pa Chanez", "age": 27 } ] }, "brec": { "cid": 898, "name": "Thao Seufert", "age": 78, "address": { "number": 3529, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Bass", "Squash", "Coffee" ], "children": [ { "name": "Classie Seufert", "age": null } ] } }
+{ "arec": { "cid": 172, "name": "Weldon Alquesta", "age": null, "address": null, "interests": [ "Music", "Fishing", "Music" ], "children": [ { "name": "Kip Alquesta", "age": null } ] }, "brec": { "cid": 961, "name": "Mirian Herpolsheimer", "age": null, "address": null, "interests": [ "Music", "Fishing", "Computers" ], "children": [ { "name": "Larissa Herpolsheimer", "age": 41 }, { "name": "Markus Herpolsheimer", "age": null }, { "name": "Natacha Herpolsheimer", "age": null } ] } }
+{ "arec": { "cid": 173, "name": "Annamae Lucien", "age": 46, "address": { "number": 1253, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Squash" ], "children": [ { "name": "Sanjuana Lucien", "age": 21 }, { "name": "Nathanael Lucien", "age": 27 }, { "name": "Jae Lucien", "age": null }, { "name": "Judith Lucien", "age": null } ] }, "brec": { "cid": 507, "name": "Yuk Flanegan", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Squash" ], "children": [ { "name": "Alexander Flanegan", "age": null } ] } }
+{ "arec": { "cid": 173, "name": "Annamae Lucien", "age": 46, "address": { "number": 1253, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Squash" ], "children": [ { "name": "Sanjuana Lucien", "age": 21 }, { "name": "Nathanael Lucien", "age": 27 }, { "name": "Jae Lucien", "age": null }, { "name": "Judith Lucien", "age": null } ] }, "brec": { "cid": 691, "name": "Sharee Charrier", "age": 17, "address": { "number": 6693, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Bass" ], "children": [ { "name": "Odessa Charrier", "age": null } ] } }
+{ "arec": { "cid": 178, "name": "Athena Kaluna", "age": null, "address": null, "interests": [ "Running", "Computers", "Basketball" ], "children": [ { "name": "Rosalba Kaluna", "age": 48 }, { "name": "Max Kaluna", "age": 10 } ] }, "brec": { "cid": 345, "name": "Derick Rippel", "age": 79, "address": { "number": 6843, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running", "Basketball", "Computers", "Basketball" ], "children": [  ] } }
+{ "arec": { "cid": 187, "name": "Seema Hartsch", "age": 80, "address": { "number": 6629, "street": "Lake St.", "city": "Portland" }, "interests": [ "Coffee", "Coffee", "Cigars" ], "children": [ { "name": "Suellen Hartsch", "age": null }, { "name": "Pennie Hartsch", "age": 20 }, { "name": "Aubrey Hartsch", "age": null }, { "name": "Randy Hartsch", "age": 32 } ] }, "brec": { "cid": 598, "name": "Venus Peat", "age": null, "address": null, "interests": [ "Coffee", "Walking", "Cigars" ], "children": [ { "name": "Antonetta Peat", "age": null }, { "name": "Shane Peat", "age": null } ] } }
+{ "arec": { "cid": 187, "name": "Seema Hartsch", "age": 80, "address": { "number": 6629, "street": "Lake St.", "city": "Portland" }, "interests": [ "Coffee", "Coffee", "Cigars" ], "children": [ { "name": "Suellen Hartsch", "age": null }, { "name": "Pennie Hartsch", "age": 20 }, { "name": "Aubrey Hartsch", "age": null }, { "name": "Randy Hartsch", "age": 32 } ] }, "brec": { "cid": 927, "name": "Lillia Hartlein", "age": 55, "address": { "number": 5856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Coffee", "Cigars" ], "children": [ { "name": "Nicky Hartlein", "age": null }, { "name": "Cassaundra Hartlein", "age": 10 }, { "name": "Micheline Hartlein", "age": 26 }, { "name": "Anton Hartlein", "age": 32 } ] } }
+{ "arec": { "cid": 198, "name": "Thelma Youkers", "age": null, "address": null, "interests": [ "Basketball", "Movies", "Cooking" ], "children": [ { "name": "Shamika Youkers", "age": 28 } ] }, "brec": { "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] } }
+{ "arec": { "cid": 207, "name": "Phyliss Honda", "age": 22, "address": { "number": 8387, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Cooking", "Music", "Books" ], "children": [ { "name": "Bee Honda", "age": null }, { "name": "Cyril Honda", "age": null }, { "name": "Vertie Honda", "age": null } ] }, "brec": { "cid": 440, "name": "Rosie Shappen", "age": null, "address": null, "interests": [ "Cooking", "Music", "Cigars" ], "children": [ { "name": "Jung Shappen", "age": 11 } ] } }
+{ "arec": { "cid": 207, "name": "Phyliss Honda", "age": 22, "address": { "number": 8387, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Cooking", "Music", "Books" ], "children": [ { "name": "Bee Honda", "age": null }, { "name": "Cyril Honda", "age": null }, { "name": "Vertie Honda", "age": null } ] }, "brec": { "cid": 825, "name": "Kirstie Rinebold", "age": 57, "address": { "number": 9463, "street": "Oak St.", "city": "Portland" }, "interests": [ "Cooking", "Cigars", "Books" ], "children": [ { "name": "Vonda Rinebold", "age": null }, { "name": "Man Rinebold", "age": 21 } ] } }
+{ "arec": { "cid": 216, "name": "Odilia Lampson", "age": null, "address": null, "interests": [ "Wine", "Databases", "Basketball" ], "children": [ { "name": "Callie Lampson", "age": null } ] }, "brec": { "cid": 220, "name": "Soila Hannemann", "age": null, "address": null, "interests": [ "Wine", "Puzzles", "Basketball" ], "children": [ { "name": "Piper Hannemann", "age": 44 } ] } }
+{ "arec": { "cid": 220, "name": "Soila Hannemann", "age": null, "address": null, "interests": [ "Wine", "Puzzles", "Basketball" ], "children": [ { "name": "Piper Hannemann", "age": 44 } ] }, "brec": { "cid": 312, "name": "Epifania Chorney", "age": 62, "address": { "number": 9749, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Puzzles", "Tennis" ], "children": [ { "name": "Lizeth Chorney", "age": 22 } ] } }
+{ "arec": { "cid": 224, "name": "Rene Rowey", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "children": [ { "name": "Necole Rowey", "age": 26 }, { "name": "Sharyl Rowey", "age": 20 }, { "name": "Yvone Rowey", "age": 36 } ] }, "brec": { "cid": 538, "name": "Mack Vollick", "age": null, "address": null, "interests": [ "Base Jumping", "Fishing", "Walking", "Computers" ], "children": [ { "name": "Gil Vollick", "age": 11 }, { "name": "Marica Vollick", "age": null } ] } }
+{ "arec": { "cid": 224, "name": "Rene Rowey", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "children": [ { "name": "Necole Rowey", "age": 26 }, { "name": "Sharyl Rowey", "age": 20 }, { "name": "Yvone Rowey", "age": 36 } ] }, "brec": { "cid": 788, "name": "Franklyn Crowner", "age": 56, "address": { "number": 4186, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Base Jumping", "Books", "Computers" ], "children": [ { "name": "Adrian Crowner", "age": 43 }, { "name": "Vasiliki Crowner", "age": null } ] } }
+{ "arec": { "cid": 237, "name": "Sona Hehn", "age": 47, "address": { "number": 3720, "street": "Oak St.", "city": "Portland" }, "interests": [ "Computers", "Squash", "Coffee" ], "children": [ { "name": "Marquerite Hehn", "age": null }, { "name": "Suellen Hehn", "age": 29 }, { "name": "Herb Hehn", "age": 29 } ] }, "brec": { "cid": 898, "name": "Thao Seufert", "age": 78, "address": { "number": 3529, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Bass", "Squash", "Coffee" ], "children": [ { "name": "Classie Seufert", "age": null } ] } }
+{ "arec": { "cid": 244, "name": "Rene Shenk", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Skiing" ], "children": [ { "name": "Victor Shenk", "age": 28 }, { "name": "Doris Shenk", "age": null }, { "name": "Max Shenk", "age": 51 } ] }, "brec": { "cid": 507, "name": "Yuk Flanegan", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Squash" ], "children": [ { "name": "Alexander Flanegan", "age": null } ] } }
+{ "arec": { "cid": 250, "name": "Angeles Saltonstall", "age": null, "address": null, "interests": [ "Tennis", "Fishing", "Movies" ], "children": [ { "name": "Suzanna Saltonstall", "age": null } ] }, "brec": { "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] } }
+{ "arec": { "cid": 250, "name": "Angeles Saltonstall", "age": null, "address": null, "interests": [ "Tennis", "Fishing", "Movies" ], "children": [ { "name": "Suzanna Saltonstall", "age": null } ] }, "brec": { "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] } }
+{ "arec": { "cid": 263, "name": "Mellisa Machalek", "age": null, "address": null, "interests": [ "Bass", "Coffee", "Skiing" ], "children": [  ] }, "brec": { "cid": 618, "name": "Janella Hurtt", "age": null, "address": null, "interests": [ "Skiing", "Coffee", "Skiing" ], "children": [ { "name": "Lupe Hurtt", "age": 17 }, { "name": "Jae Hurtt", "age": 14 }, { "name": "Evan Hurtt", "age": 45 } ] } }
+{ "arec": { "cid": 264, "name": "Leon Yoshizawa", "age": 81, "address": { "number": 608, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Running", "Books", "Running" ], "children": [ { "name": "Carmela Yoshizawa", "age": 34 } ] }, "brec": { "cid": 804, "name": "Joaquina Burlin", "age": 77, "address": { "number": 5479, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Running", "Wine", "Running" ], "children": [  ] } }
+{ "arec": { "cid": 268, "name": "Fernando Pingel", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Books" ], "children": [ { "name": "Latrice Pingel", "age": null }, { "name": "Wade Pingel", "age": 13 }, { "name": "Christal Pingel", "age": null }, { "name": "Melania Pingel", "age": null } ] }, "brec": { "cid": 446, "name": "Lilly Grannell", "age": 21, "address": { "number": 5894, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Computers", "Tennis", "Puzzles", "Books" ], "children": [ { "name": "Victor Grannell", "age": null } ] } }
+{ "arec": { "cid": 273, "name": "Corrinne Seaquist", "age": 24, "address": { "number": 6712, "street": "7th St.", "city": "Portland" }, "interests": [ "Puzzles", "Coffee", "Wine" ], "children": [ { "name": "Mignon Seaquist", "age": null }, { "name": "Leo Seaquist", "age": null } ] }, "brec": { "cid": 709, "name": "Jazmine Twiddy", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Wine" ], "children": [ { "name": "Veronika Twiddy", "age": 21 } ] } }
+{ "arec": { "cid": 274, "name": "Claude Harral", "age": null, "address": null, "interests": [ "Squash", "Bass", "Cooking" ], "children": [ { "name": "Archie Harral", "age": null }, { "name": "Royal Harral", "age": null } ] }, "brec": { "cid": 654, "name": "Louis Laubersheimer", "age": 76, "address": { "number": 8010, "street": "7th St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Bass", "Cooking" ], "children": [ { "name": "Jewel Laubersheimer", "age": 22 }, { "name": "Toccara Laubersheimer", "age": 45 }, { "name": "Eve Laubersheimer", "age": null } ] } }
+{ "arec": { "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] }, "brec": { "cid": 892, "name": "Madge Hendson", "age": 79, "address": { "number": 8832, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Fishing", "Skiing" ], "children": [ { "name": "Elia Hendson", "age": 48 }, { "name": "Lashawn Hendson", "age": 27 } ] } }
+{ "arec": { "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] }, "brec": { "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] } }
+{ "arec": { "cid": 297, "name": "Adeline Frierson", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Fishing" ], "children": [ { "name": "Marci Frierson", "age": null }, { "name": "Rolanda Frierson", "age": null }, { "name": "Del Frierson", "age": null } ] }, "brec": { "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] } }
+{ "arec": { "cid": 297, "name": "Adeline Frierson", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Fishing" ], "children": [ { "name": "Marci Frierson", "age": null }, { "name": "Rolanda Frierson", "age": null }, { "name": "Del Frierson", "age": null } ] }, "brec": { "cid": 996, "name": "Elouise Wider", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Base Jumping" ], "children": [  ] } }
+{ "arec": { "cid": 299, "name": "Jacob Wainman", "age": 76, "address": { "number": 4551, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Wine", "Coffee" ], "children": [ { "name": "Abram Wainman", "age": 28 }, { "name": "Ramonita Wainman", "age": 18 }, { "name": "Sheryll Wainman", "age": null } ] }, "brec": { "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] } }
+{ "arec": { "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] }, "brec": { "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] } }
+{ "arec": { "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] }, "brec": { "cid": 661, "name": "Lorita Kraut", "age": 43, "address": { "number": 5017, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Movies", "Bass" ], "children": [ { "name": "Mirian Kraut", "age": null } ] } }
+{ "arec": { "cid": 312, "name": "Epifania Chorney", "age": 62, "address": { "number": 9749, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Puzzles", "Tennis" ], "children": [ { "name": "Lizeth Chorney", "age": 22 } ] }, "brec": { "cid": 895, "name": "Joie Siffert", "age": null, "address": null, "interests": [ "Wine", "Skiing", "Puzzles", "Tennis" ], "children": [ { "name": "Erma Siffert", "age": null }, { "name": "Natosha Siffert", "age": 38 }, { "name": "Somer Siffert", "age": 27 } ] } }
+{ "arec": { "cid": 326, "name": "Tad Tellers", "age": null, "address": null, "interests": [ "Books", "Tennis", "Base Jumping" ], "children": [ { "name": "Fannie Tellers", "age": null } ] }, "brec": { "cid": 541, "name": "Sammy Adamitis", "age": 71, "address": { "number": 5593, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Books", "Tennis", "Cooking" ], "children": [  ] } }
+{ "arec": { "cid": 335, "name": "Odessa Dammeyer", "age": 18, "address": { "number": 6828, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Basketball", "Bass", "Cigars" ], "children": [ { "name": "Lindsey Dammeyer", "age": null } ] }, "brec": { "cid": 660, "name": "Israel Aday", "age": null, "address": null, "interests": [ "Wine", "Bass", "Cigars" ], "children": [ { "name": "Mi Aday", "age": null } ] } }
+{ "arec": { "cid": 352, "name": "Bonny Sischo", "age": null, "address": null, "interests": [ "Bass", "Movies", "Computers" ], "children": [ { "name": "Judith Sischo", "age": 43 }, { "name": "Adeline Sischo", "age": null }, { "name": "Dayna Sischo", "age": null } ] }, "brec": { "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] } }
+{ "arec": { "cid": 352, "name": "Bonny Sischo", "age": null, "address": null, "interests": [ "Bass", "Movies", "Computers" ], "children": [ { "name": "Judith Sischo", "age": 43 }, { "name": "Adeline Sischo", "age": null }, { "name": "Dayna Sischo", "age": null } ] }, "brec": { "cid": 909, "name": "Mariko Sharar", "age": null, "address": null, "interests": [ "Squash", "Movies", "Computers" ], "children": [  ] } }
+{ "arec": { "cid": 359, "name": "Sharika Vientos", "age": 42, "address": { "number": 5981, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Walking", "Bass", "Fishing", "Movies" ], "children": [ { "name": "Clifton Vientos", "age": 21 }, { "name": "Renae Vientos", "age": null }, { "name": "Marcelo Vientos", "age": 31 }, { "name": "Jacalyn Vientos", "age": null } ] }, "brec": { "cid": 969, "name": "Laurinda Gnerre", "age": 42, "address": { "number": 2284, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Walking", "Bass", "Fishing", "Video Games" ], "children": [ { "name": "Veronica Gnerre", "age": null } ] } }
+{ "arec": { "cid": 363, "name": "Merlene Hoying", "age": 25, "address": { "number": 2105, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash", "Music" ], "children": [ { "name": "Andrew Hoying", "age": 10 } ] }, "brec": { "cid": 415, "name": "Valentin Mclarney", "age": null, "address": null, "interests": [ "Squash", "Squash", "Video Games" ], "children": [ { "name": "Vanda Mclarney", "age": 17 } ] } }
+{ "arec": { "cid": 363, "name": "Merlene Hoying", "age": 25, "address": { "number": 2105, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash", "Music" ], "children": [ { "name": "Andrew Hoying", "age": 10 } ] }, "brec": { "cid": 642, "name": "Odell Nova", "age": 25, "address": { "number": 896, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Squash", "Music" ], "children": [ { "name": "Leopoldo Nova", "age": null }, { "name": "Rickey Nova", "age": null }, { "name": "Mike Nova", "age": 14 }, { "name": "Tamie Nova", "age": 14 } ] } }
+{ "arec": { "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] }, "brec": { "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] } }
+{ "arec": { "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] }, "brec": { "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] } }
+{ "arec": { "cid": 378, "name": "Melany Matias", "age": 10, "address": { "number": 8838, "street": "Main St.", "city": "Seattle" }, "interests": [ "Coffee", "Tennis", "Bass" ], "children": [ { "name": "Earnestine Matias", "age": null }, { "name": "Lore Matias", "age": null } ] }, "brec": { "cid": 580, "name": "Liana Gabbert", "age": null, "address": null, "interests": [ "Coffee", "Tennis", "Bass", "Running" ], "children": [  ] } }
+{ "arec": { "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] }, "brec": { "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] } }
+{ "arec": { "cid": 397, "name": "Blake Kealy", "age": 34, "address": { "number": 2156, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Databases", "Wine", "Cigars" ], "children": [ { "name": "Lorenza Kealy", "age": null }, { "name": "Beula Kealy", "age": 15 }, { "name": "Kristofer Kealy", "age": null }, { "name": "Shayne Kealy", "age": null } ] }, "brec": { "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] } }
+{ "arec": { "cid": 402, "name": "Terrilyn Shinall", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Music" ], "children": [ { "name": "Minh Shinall", "age": null }, { "name": "Diedre Shinall", "age": 22 } ] }, "brec": { "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] } }
+{ "arec": { "cid": 406, "name": "Addie Mandez", "age": null, "address": null, "interests": [ "Tennis", "Cigars", "Books" ], "children": [ { "name": "Rosendo Mandez", "age": 34 } ] }, "brec": { "cid": 489, "name": "Brigid Delosier", "age": 31, "address": { "number": 6082, "street": "Oak St.", "city": "Portland" }, "interests": [ "Tennis", "Cigars", "Music" ], "children": [ { "name": "Allegra Delosier", "age": null }, { "name": "Yong Delosier", "age": 10 }, { "name": "Steffanie Delosier", "age": 13 } ] } }
+{ "arec": { "cid": 406, "name": "Addie Mandez", "age": null, "address": null, "interests": [ "Tennis", "Cigars", "Books" ], "children": [ { "name": "Rosendo Mandez", "age": 34 } ] }, "brec": { "cid": 825, "name": "Kirstie Rinebold", "age": 57, "address": { "number": 9463, "street": "Oak St.", "city": "Portland" }, "interests": [ "Cooking", "Cigars", "Books" ], "children": [ { "name": "Vonda Rinebold", "age": null }, { "name": "Man Rinebold", "age": 21 } ] } }
+{ "arec": { "cid": 412, "name": "Devon Szalai", "age": 26, "address": { "number": 2384, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books", "Books" ], "children": [ { "name": "Yolonda Szalai", "age": null }, { "name": "Denita Szalai", "age": null }, { "name": "Priscila Szalai", "age": 10 }, { "name": "Cassondra Szalai", "age": 12 } ] }, "brec": { "cid": 722, "name": "Noel Goncalves", "age": null, "address": null, "interests": [ "Books", "Bass", "Books", "Books" ], "children": [ { "name": "Latrice Goncalves", "age": null }, { "name": "Evelia Goncalves", "age": 36 }, { "name": "Etta Goncalves", "age": 11 }, { "name": "Collin Goncalves", "age": null } ] } }
+{ "arec": { "cid": 417, "name": "Irene Funderberg", "age": 45, "address": { "number": 8503, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Skiing", "Running" ], "children": [ { "name": "Lyndia Funderberg", "age": 14 }, { "name": "Herta Funderberg", "age": null } ] }, "brec": { "cid": 629, "name": "Mayola Clabo", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Running" ], "children": [ { "name": "Rigoberto Clabo", "age": 58 } ] } }
+{ "arec": { "cid": 417, "name": "Irene Funderberg", "age": 45, "address": { "number": 8503, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Skiing", "Running" ], "children": [ { "name": "Lyndia Funderberg", "age": 14 }, { "name": "Herta Funderberg", "age": null } ] }, "brec": { "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] } }
+{ "arec": { "cid": 418, "name": "Gavin Delpino", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Gianna Delpino", "age": null }, { "name": "Carmella Delpino", "age": 55 } ] }, "brec": { "cid": 621, "name": "Theresa Satterthwaite", "age": 16, "address": { "number": 3249, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Rickie Satterthwaite", "age": null }, { "name": "Rina Satterthwaite", "age": null } ] } }
+{ "arec": { "cid": 429, "name": "Eladia Scannell", "age": 20, "address": { "number": 5036, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Music", "Movies" ], "children": [  ] }, "brec": { "cid": 518, "name": "Cora Ingargiola", "age": null, "address": null, "interests": [ "Skiing", "Squash", "Movies" ], "children": [ { "name": "Katlyn Ingargiola", "age": null }, { "name": "Mike Ingargiola", "age": null }, { "name": "Lawrence Ingargiola", "age": null }, { "name": "Isabelle Ingargiola", "age": null } ] } }
+{ "arec": { "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }, "brec": { "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] } }
+{ "arec": { "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }, "brec": { "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] } }
+{ "arec": { "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }, "brec": { "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] } }
+{ "arec": { "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }, "brec": { "cid": 859, "name": "Mozelle Catillo", "age": 61, "address": { "number": 253, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Databases", "Cooking", "Wine" ], "children": [  ] } }
+{ "arec": { "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }, "brec": { "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] } }
+{ "arec": { "cid": 438, "name": "Allegra Pefanis", "age": null, "address": null, "interests": [ "Computers", "Music", "Cigars" ], "children": [  ] }, "brec": { "cid": 440, "name": "Rosie Shappen", "age": null, "address": null, "interests": [ "Cooking", "Music", "Cigars" ], "children": [ { "name": "Jung Shappen", "age": 11 } ] } }
+{ "arec": { "cid": 444, "name": "Demetra Sava", "age": null, "address": null, "interests": [ "Music", "Fishing", "Databases", "Wine" ], "children": [ { "name": "Fidel Sava", "age": 16 } ] }, "brec": { "cid": 484, "name": "Bennie Dragaj", "age": null, "address": null, "interests": [ "Fishing", "Databases", "Wine" ], "children": [ { "name": "Viva Dragaj", "age": 13 } ] } }
+{ "arec": { "cid": 445, "name": "Walton Komo", "age": 16, "address": { "number": 8769, "street": "Main St.", "city": "Seattle" }, "interests": [ "Running", "Basketball", "Tennis" ], "children": [  ] }, "brec": { "cid": 828, "name": "Marcelle Steinhour", "age": null, "address": null, "interests": [ "Running", "Basketball", "Walking" ], "children": [ { "name": "Jimmie Steinhour", "age": 13 }, { "name": "Kirstie Steinhour", "age": 19 } ] } }
+{ "arec": { "cid": 445, "name": "Walton Komo", "age": 16, "address": { "number": 8769, "street": "Main St.", "city": "Seattle" }, "interests": [ "Running", "Basketball", "Tennis" ], "children": [  ] }, "brec": { "cid": 962, "name": "Taryn Coley", "age": null, "address": null, "interests": [ "Running", "Basketball", "Cooking" ], "children": [  ] } }
+{ "arec": { "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] }, "brec": { "cid": 927, "name": "Lillia Hartlein", "age": 55, "address": { "number": 5856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Coffee", "Cigars" ], "children": [ { "name": "Nicky Hartlein", "age": null }, { "name": "Cassaundra Hartlein", "age": 10 }, { "name": "Micheline Hartlein", "age": 26 }, { "name": "Anton Hartlein", "age": 32 } ] } }
+{ "arec": { "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] }, "brec": { "cid": 734, "name": "Lera Korn", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Cigars" ], "children": [ { "name": "Criselda Korn", "age": 37 } ] } }
+{ "arec": { "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] }, "brec": { "cid": 791, "name": "Jame Apresa", "age": 66, "address": { "number": 8417, "street": "Main St.", "city": "San Jose" }, "interests": [ "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Awilda Apresa", "age": null }, { "name": "Nelle Apresa", "age": 40 }, { "name": "Terrell Apresa", "age": null }, { "name": "Malia Apresa", "age": 43 } ] } }
+{ "arec": { "cid": 455, "name": "Manual Altizer", "age": 70, "address": { "number": 6293, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Fishing", "Coffee" ], "children": [ { "name": "Katherine Altizer", "age": null } ] }, "brec": { "cid": 488, "name": "Dannielle Wilkie", "age": null, "address": null, "interests": [ "Running", "Fishing", "Coffee", "Basketball" ], "children": [ { "name": "Vita Wilkie", "age": 17 }, { "name": "Marisa Wilkie", "age": null }, { "name": "Faustino Wilkie", "age": null } ] } }
+{ "arec": { "cid": 473, "name": "Cordell Solas", "age": null, "address": null, "interests": [ "Squash", "Music", "Bass", "Puzzles" ], "children": [ { "name": "Douglass Solas", "age": null }, { "name": "Claribel Solas", "age": null }, { "name": "Fred Solas", "age": null }, { "name": "Ahmed Solas", "age": 21 } ] }, "brec": { "cid": 527, "name": "Lance Kenison", "age": 77, "address": { "number": 8750, "street": "Main St.", "city": "San Jose" }, "interests": [ "Squash", "Cooking", "Bass", "Puzzles" ], "children": [ { "name": "Youlanda Kenison", "age": null }, { "name": "Lavon Kenison", "age": null }, { "name": "Maryann Kenison", "age": 60 }, { "name": "Kecia Kenison", "age": 50 } ] } }
+{ "arec": { "cid": 484, "name": "Bennie Dragaj", "age": null, "address": null, "interests": [ "Fishing", "Databases", "Wine" ], "children": [ { "name": "Viva Dragaj", "age": 13 } ] }, "brec": { "cid": 986, "name": "Tennille Wikle", "age": 78, "address": { "number": 3428, "street": "View St.", "city": "Portland" }, "interests": [ "Movies", "Databases", "Wine" ], "children": [ { "name": "Lourie Wikle", "age": null }, { "name": "Laure Wikle", "age": null } ] } }
+{ "arec": { "cid": 487, "name": "Zenia Virgilio", "age": 46, "address": { "number": 584, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Walking", "Squash", "Wine" ], "children": [ { "name": "Quintin Virgilio", "age": null }, { "name": "Edith Virgilio", "age": null }, { "name": "Nicolle Virgilio", "age": 33 } ] }, "brec": { "cid": 735, "name": "Lonnie Bechel", "age": 36, "address": { "number": 592, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Walking", "Cigars", "Squash", "Wine" ], "children": [  ] } }
+{ "arec": { "cid": 496, "name": "Lonna Starkweather", "age": 80, "address": { "number": 1162, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Bass", "Running" ], "children": [ { "name": "Matilda Starkweather", "age": null } ] }, "brec": { "cid": 545, "name": "Dolores Ferer", "age": null, "address": null, "interests": [ "Coffee", "Bass", "Tennis" ], "children": [ { "name": "Bridgette Ferer", "age": null } ] } }
+{ "arec": { "cid": 496, "name": "Lonna Starkweather", "age": 80, "address": { "number": 1162, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Bass", "Running" ], "children": [ { "name": "Matilda Starkweather", "age": null } ] }, "brec": { "cid": 580, "name": "Liana Gabbert", "age": null, "address": null, "interests": [ "Coffee", "Tennis", "Bass", "Running" ], "children": [  ] } }
+{ "arec": { "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }, "brec": { "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] } }
+{ "arec": { "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }, "brec": { "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] } }
+{ "arec": { "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }, "brec": { "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] } }
+{ "arec": { "cid": 522, "name": "Daryl Kissack", "age": 86, "address": { "number": 7825, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Squash", "Base Jumping", "Tennis" ], "children": [ { "name": "Darrel Kissack", "age": 21 } ] }, "brec": { "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] } }
+{ "arec": { "cid": 522, "name": "Daryl Kissack", "age": 86, "address": { "number": 7825, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Squash", "Base Jumping", "Tennis" ], "children": [ { "name": "Darrel Kissack", "age": 21 } ] }, "brec": { "cid": 812, "name": "Bee Godette", "age": 26, "address": { "number": 1757, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Base Jumping", "Tennis" ], "children": [ { "name": "Madaline Godette", "age": 10 }, { "name": "Shasta Godette", "age": 15 }, { "name": "Parthenia Godette", "age": 11 }, { "name": "Priscila Godette", "age": 13 } ] } }
+{ "arec": { "cid": 537, "name": "Mara Hugar", "age": null, "address": null, "interests": [ "Fishing", "Skiing", "Skiing" ], "children": [ { "name": "Krista Hugar", "age": null } ] }, "brec": { "cid": 600, "name": "Cordell Sherburn", "age": null, "address": null, "interests": [ "Squash", "Skiing", "Skiing" ], "children": [ { "name": "Shenna Sherburn", "age": 22 }, { "name": "Minna Sherburn", "age": 10 }, { "name": "Tari Sherburn", "age": null } ] } }
+{ "arec": { "cid": 541, "name": "Sammy Adamitis", "age": 71, "address": { "number": 5593, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Books", "Tennis", "Cooking" ], "children": [  ] }, "brec": { "cid": 913, "name": "Evelynn Fague", "age": 42, "address": { "number": 5729, "street": "7th St.", "city": "Seattle" }, "interests": [ "Books", "Databases", "Cooking" ], "children": [  ] } }
+{ "arec": { "cid": 545, "name": "Dolores Ferer", "age": null, "address": null, "interests": [ "Coffee", "Bass", "Tennis" ], "children": [ { "name": "Bridgette Ferer", "age": null } ] }, "brec": { "cid": 566, "name": "Asley Grow", "age": null, "address": null, "interests": [ "Coffee", "Books", "Tennis" ], "children": [ { "name": "Dale Grow", "age": null } ] } }
+{ "arec": { "cid": 562, "name": "Etta Hooton", "age": null, "address": null, "interests": [ "Databases", "Cigars", "Music", "Video Games" ], "children": [ { "name": "Sherice Hooton", "age": null }, { "name": "Estefana Hooton", "age": 38 }, { "name": "Nidia Hooton", "age": 47 }, { "name": "Erwin Hooton", "age": null } ] }, "brec": { "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] } }
+{ "arec": { "cid": 563, "name": "Deirdre Landero", "age": null, "address": null, "interests": [ "Books", "Fishing", "Video Games" ], "children": [ { "name": "Norman Landero", "age": 59 }, { "name": "Jennine Landero", "age": 45 }, { "name": "Rutha Landero", "age": 19 }, { "name": "Jackie Landero", "age": 29 } ] }, "brec": { "cid": 941, "name": "Jamey Jakobson", "age": null, "address": null, "interests": [ "Books", "Cooking", "Video Games" ], "children": [ { "name": "Elmer Jakobson", "age": 14 }, { "name": "Minh Jakobson", "age": 30 } ] } }
+{ "arec": { "cid": 564, "name": "Inger Dargin", "age": 56, "address": { "number": 8704, "street": "View St.", "city": "Mountain View" }, "interests": [ "Wine", "Running", "Computers" ], "children": [  ] }, "brec": { "cid": 849, "name": "Kristen Zapalac", "age": 14, "address": { "number": 4087, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Cooking", "Running", "Computers" ], "children": [  ] } }
+{ "arec": { "cid": 566, "name": "Asley Grow", "age": null, "address": null, "interests": [ "Coffee", "Books", "Tennis" ], "children": [ { "name": "Dale Grow", "age": null } ] }, "brec": { "cid": 750, "name": "Rosaura Gaul", "age": null, "address": null, "interests": [ "Music", "Books", "Tennis" ], "children": [ { "name": "Letisha Gaul", "age": 41 } ] } }
+{ "arec": { "cid": 575, "name": "Phyliss Mattes", "age": 26, "address": { "number": 3956, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Music", "Running", "Music" ], "children": [  ] }, "brec": { "cid": 757, "name": "Bertie Flemming", "age": null, "address": null, "interests": [ "Tennis", "Music", "Running", "Cooking" ], "children": [ { "name": "Temeka Flemming", "age": 46 }, { "name": "Terrance Flemming", "age": null }, { "name": "Jenette Flemming", "age": 23 }, { "name": "Debra Flemming", "age": null } ] } }
+{ "arec": { "cid": 585, "name": "Young Drube", "age": 21, "address": { "number": 6960, "street": "View St.", "city": "Seattle" }, "interests": [ "Basketball", "Fishing", "Walking" ], "children": [ { "name": "Irwin Drube", "age": null }, { "name": "Gustavo Drube", "age": null } ] }, "brec": { "cid": 808, "name": "Brande Decius", "age": null, "address": null, "interests": [ "Basketball", "Fishing", "Puzzles" ], "children": [ { "name": "Li Decius", "age": 56 }, { "name": "Eusebio Decius", "age": 50 }, { "name": "Clementina Decius", "age": 29 } ] } }
+{ "arec": { "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] }, "brec": { "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] } }
+{ "arec": { "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] }, "brec": { "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] } }
+{ "arec": { "cid": 588, "name": "Debora Laughinghouse", "age": 87, "address": { "number": 5099, "street": "View St.", "city": "San Jose" }, "interests": [ "Tennis", "Walking", "Databases" ], "children": [ { "name": "Frederica Laughinghouse", "age": 59 }, { "name": "Johnie Laughinghouse", "age": 12 }, { "name": "Numbers Laughinghouse", "age": 73 } ] }, "brec": { "cid": 853, "name": "Denisse Peralto", "age": 25, "address": { "number": 3931, "street": "7th St.", "city": "Portland" }, "interests": [ "Tennis", "Walking", "Basketball" ], "children": [ { "name": "Asha Peralto", "age": 14 }, { "name": "Clark Peralto", "age": null }, { "name": "Jessika Peralto", "age": null }, { "name": "Nadene Peralto", "age": null } ] } }
+{ "arec": { "cid": 600, "name": "Cordell Sherburn", "age": null, "address": null, "interests": [ "Squash", "Skiing", "Skiing" ], "children": [ { "name": "Shenna Sherburn", "age": 22 }, { "name": "Minna Sherburn", "age": 10 }, { "name": "Tari Sherburn", "age": null } ] }, "brec": { "cid": 703, "name": "Susanne Pettey", "age": null, "address": null, "interests": [ "Squash", "Basketball", "Skiing" ], "children": [ { "name": "Nancey Pettey", "age": 35 }, { "name": "Lawana Pettey", "age": null }, { "name": "Percy Pettey", "age": 25 } ] } }
+{ "arec": { "cid": 602, "name": "Clyde Salada", "age": 59, "address": { "number": 8316, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Movies", "Skiing", "Cooking" ], "children": [  ] }, "brec": { "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] } }
+{ "arec": { "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] }, "brec": { "cid": 639, "name": "Zena Seehusen", "age": 24, "address": { "number": 6303, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Cooking", "Movies", "Music" ], "children": [ { "name": "Hester Seehusen", "age": null }, { "name": "Coreen Seehusen", "age": 12 } ] } }
+{ "arec": { "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] }, "brec": { "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] } }
+{ "arec": { "cid": 621, "name": "Theresa Satterthwaite", "age": 16, "address": { "number": 3249, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Rickie Satterthwaite", "age": null }, { "name": "Rina Satterthwaite", "age": null } ] }, "brec": { "cid": 929, "name": "Jean Guitierrez", "age": 75, "address": { "number": 9736, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Wine", "Wine", "Fishing" ], "children": [  ] } }
+{ "arec": { "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] }, "brec": { "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] } }
+{ "arec": { "cid": 629, "name": "Mayola Clabo", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Running" ], "children": [ { "name": "Rigoberto Clabo", "age": 58 } ] }, "brec": { "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] } }
+{ "arec": { "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] }, "brec": { "cid": 750, "name": "Rosaura Gaul", "age": null, "address": null, "interests": [ "Music", "Books", "Tennis" ], "children": [ { "name": "Letisha Gaul", "age": 41 } ] } }
+{ "arec": { "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] }, "brec": { "cid": 812, "name": "Bee Godette", "age": 26, "address": { "number": 1757, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Base Jumping", "Tennis" ], "children": [ { "name": "Madaline Godette", "age": 10 }, { "name": "Shasta Godette", "age": 15 }, { "name": "Parthenia Godette", "age": 11 }, { "name": "Priscila Godette", "age": 13 } ] } }
+{ "arec": { "cid": 636, "name": "Babara Shore", "age": 83, "address": { "number": 9452, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Candy Shore", "age": 58 }, { "name": "Nanci Shore", "age": null }, { "name": "Asia Shore", "age": null } ] }, "brec": { "cid": 992, "name": "Staci Alexandropoul", "age": null, "address": null, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Casimira Alexandropoul", "age": null }, { "name": "Kena Alexandropoul", "age": 54 }, { "name": "Ellie Alexandropoul", "age": null }, { "name": "Ambrose Alexandropoul", "age": null } ] } }
+{ "arec": { "cid": 649, "name": "Anisha Sender", "age": null, "address": null, "interests": [ "Tennis", "Databases", "Bass" ], "children": [ { "name": "Viva Sender", "age": 40 }, { "name": "Terica Sender", "age": null } ] }, "brec": { "cid": 661, "name": "Lorita Kraut", "age": 43, "address": { "number": 5017, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Movies", "Bass" ], "children": [ { "name": "Mirian Kraut", "age": null } ] } }
+{ "arec": { "cid": 649, "name": "Anisha Sender", "age": null, "address": null, "interests": [ "Tennis", "Databases", "Bass" ], "children": [ { "name": "Viva Sender", "age": 40 }, { "name": "Terica Sender", "age": null } ] }, "brec": { "cid": 928, "name": "Maddie Diclaudio", "age": 33, "address": { "number": 4674, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Databases", "Bass" ], "children": [ { "name": "Dominique Diclaudio", "age": 12 } ] } }
+{ "arec": { "cid": 655, "name": "Shaun Brandenburg", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Base Jumping" ], "children": [ { "name": "Ned Brandenburg", "age": null }, { "name": "Takako Brandenburg", "age": 41 }, { "name": "Astrid Brandenburg", "age": null }, { "name": "Patience Brandenburg", "age": null } ] }, "brec": { "cid": 996, "name": "Elouise Wider", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Base Jumping" ], "children": [  ] } }
+{ "arec": { "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [  ] }, "brec": { "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] } }
+{ "arec": { "cid": 662, "name": "Domonique Corbi", "age": 13, "address": { "number": 7286, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Tennis", "Cooking", "Computers" ], "children": [ { "name": "Katrice Corbi", "age": null }, { "name": "Idalia Corbi", "age": null }, { "name": "Hayley Corbi", "age": null } ] }, "brec": { "cid": 964, "name": "Stephany Soders", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Computers" ], "children": [  ] } }
+{ "arec": { "cid": 670, "name": "Angelo Kellar", "age": 22, "address": { "number": 3178, "street": "View St.", "city": "Seattle" }, "interests": [ "Wine", "Music", "Fishing" ], "children": [ { "name": "Zula Kellar", "age": null }, { "name": "Brittaney Kellar", "age": 10 }, { "name": "Fredia Kellar", "age": null } ] }, "brec": { "cid": 929, "name": "Jean Guitierrez", "age": 75, "address": { "number": 9736, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Wine", "Wine", "Fishing" ], "children": [  ] } }
+{ "arec": { "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] }, "brec": { "cid": 916, "name": "Kris Mcmarlin", "age": null, "address": null, "interests": [ "Movies", "Music", "Puzzles" ], "children": [  ] } }
+{ "arec": { "cid": 704, "name": "Melodee Clemons", "age": null, "address": null, "interests": [ "Base Jumping", "Tennis", "Video Games" ], "children": [ { "name": "Doreatha Clemons", "age": 22 } ] }, "brec": { "cid": 901, "name": "Riva Ziko", "age": null, "address": null, "interests": [ "Running", "Tennis", "Video Games" ], "children": [ { "name": "Leandra Ziko", "age": 49 }, { "name": "Torrie Ziko", "age": null } ] } }
+{ "arec": { "cid": 704, "name": "Melodee Clemons", "age": null, "address": null, "interests": [ "Base Jumping", "Tennis", "Video Games" ], "children": [ { "name": "Doreatha Clemons", "age": 22 } ] }, "brec": { "cid": 948, "name": "Thad Scialpi", "age": 22, "address": { "number": 8731, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Tennis", "Wine" ], "children": [ { "name": "Harlan Scialpi", "age": 10 }, { "name": "Lucile Scialpi", "age": 11 }, { "name": "Audria Scialpi", "age": null } ] } }
+{ "arec": { "cid": 710, "name": "Arlen Horka", "age": null, "address": null, "interests": [ "Movies", "Coffee", "Walking" ], "children": [ { "name": "Valencia Horka", "age": null }, { "name": "Wesley Horka", "age": null } ] }, "brec": { "cid": 923, "name": "Bobbi Ursino", "age": null, "address": null, "interests": [ "Movies", "Books", "Walking" ], "children": [ { "name": "Shon Ursino", "age": null }, { "name": "Lorean Ursino", "age": null } ] } }
+{ "arec": { "cid": 744, "name": "Crysta Christen", "age": 57, "address": { "number": 439, "street": "Hill St.", "city": "Portland" }, "interests": [ "Basketball", "Squash", "Base Jumping" ], "children": [  ] }, "brec": { "cid": 856, "name": "Inocencia Petzold", "age": 83, "address": { "number": 4631, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Basketball", "Squash", "Movies", "Base Jumping" ], "children": [  ] } }
+{ "arec": { "cid": 769, "name": "Isaias Tenny", "age": 71, "address": { "number": 270, "street": "Park St.", "city": "Portland" }, "interests": [ "Wine", "Fishing", "Base Jumping" ], "children": [ { "name": "Theo Tenny", "age": null }, { "name": "Shena Tenny", "age": null }, { "name": "Coralee Tenny", "age": null }, { "name": "Orval Tenny", "age": 39 } ] }, "brec": { "cid": 848, "name": "Myrta Kopf", "age": null, "address": null, "interests": [ "Wine", "Basketball", "Base Jumping" ], "children": [  ] } }
+{ "arec": { "cid": 776, "name": "Dagmar Sarkis", "age": null, "address": null, "interests": [ "Basketball", "Running", "Wine" ], "children": [ { "name": "Tari Sarkis", "age": null }, { "name": "Rana Sarkis", "age": 56 }, { "name": "Merissa Sarkis", "age": null }, { "name": "Lori Sarkis", "age": 26 } ] }, "brec": { "cid": 908, "name": "Ferdinand Auila", "age": 82, "address": { "number": 1071, "street": "Lake St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running", "Wine" ], "children": [ { "name": "Ai Auila", "age": 69 }, { "name": "Laurel Auila", "age": null } ] } }
+{ "arec": { "cid": 791, "name": "Jame Apresa", "age": 66, "address": { "number": 8417, "street": "Main St.", "city": "San Jose" }, "interests": [ "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Awilda Apresa", "age": null }, { "name": "Nelle Apresa", "age": 40 }, { "name": "Terrell Apresa", "age": null }, { "name": "Malia Apresa", "age": 43 } ] }, "brec": { "cid": 801, "name": "Julio Brun", "age": 13, "address": { "number": 9774, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Puzzles", "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Peter Brun", "age": null }, { "name": "Remona Brun", "age": null }, { "name": "Giovanni Brun", "age": null } ] } }
+{ "arec": { "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] }, "brec": { "cid": 861, "name": "Hugh Mcbrien", "age": null, "address": null, "interests": [ "Skiing", "Cigars", "Cooking" ], "children": [ { "name": "Otha Mcbrien", "age": 38 } ] } }
+{ "arec": { "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] }, "brec": { "cid": 867, "name": "Denise Dipiero", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking", "Running" ], "children": [ { "name": "Santa Dipiero", "age": null } ] } }
+{ "arec": { "cid": 828, "name": "Marcelle Steinhour", "age": null, "address": null, "interests": [ "Running", "Basketball", "Walking" ], "children": [ { "name": "Jimmie Steinhour", "age": 13 }, { "name": "Kirstie Steinhour", "age": 19 } ] }, "brec": { "cid": 962, "name": "Taryn Coley", "age": null, "address": null, "interests": [ "Running", "Basketball", "Cooking" ], "children": [  ] } }
+{ "arec": { "cid": 853, "name": "Denisse Peralto", "age": 25, "address": { "number": 3931, "street": "7th St.", "city": "Portland" }, "interests": [ "Tennis", "Walking", "Basketball" ], "children": [ { "name": "Asha Peralto", "age": 14 }, { "name": "Clark Peralto", "age": null }, { "name": "Jessika Peralto", "age": null }, { "name": "Nadene Peralto", "age": null } ] }, "brec": { "cid": 912, "name": "Alessandra Kaskey", "age": 52, "address": { "number": 6906, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Skiing", "Walking", "Basketball" ], "children": [ { "name": "Mack Kaskey", "age": null } ] } }
+{ "arec": { "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] }, "brec": { "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] } }
+{ "arec": { "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] }, "brec": { "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] } }
+{ "arec": { "cid": 859, "name": "Mozelle Catillo", "age": 61, "address": { "number": 253, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Databases", "Cooking", "Wine" ], "children": [  ] }, "brec": { "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] } }
+{ "arec": { "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] }, "brec": { "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] } }
+{ "arec": { "cid": 892, "name": "Madge Hendson", "age": 79, "address": { "number": 8832, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Fishing", "Skiing" ], "children": [ { "name": "Elia Hendson", "age": 48 }, { "name": "Lashawn Hendson", "age": 27 } ] }, "brec": { "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] } }
+{ "arec": { "cid": 908, "name": "Ferdinand Auila", "age": 82, "address": { "number": 1071, "street": "Lake St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running", "Wine" ], "children": [ { "name": "Ai Auila", "age": 69 }, { "name": "Laurel Auila", "age": null } ] }, "brec": { "cid": 948, "name": "Thad Scialpi", "age": 22, "address": { "number": 8731, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Tennis", "Wine" ], "children": [ { "name": "Harlan Scialpi", "age": 10 }, { "name": "Lucile Scialpi", "age": 11 }, { "name": "Audria Scialpi", "age": null } ] } }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-jaccard.adm
new file mode 100644
index 0000000..b733c588
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-olist-jaccard.adm
@@ -0,0 +1,267 @@
+{ "a": [ "Bass", "Tennis", "Bass", "Cooking" ], "b": [ "Bass", "Cooking", "Running", "Tennis" ] }
+{ "a": [ "Bass", "Wine" ], "b": [ "Bass", "Wine" ] }
+{ "a": [ "Music", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Music", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Music", "Databases" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Music", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Wine", "Walking" ], "b": [ "Wine", "Walking" ] }
+{ "a": [ "Wine", "Walking" ], "b": [ "Walking", "Wine" ] }
+{ "a": [ "Base Jumping", "Cigars", "Movies" ], "b": [ "Base Jumping", "Cigars", "Movies" ] }
+{ "a": [ "Fishing", "Running", "Tennis", "Running" ], "b": [ "Tennis", "Coffee", "Running", "Fishing" ] }
+{ "a": [ "Skiing", "Walking" ], "b": [ "Skiing", "Walking" ] }
+{ "a": [ "Base Jumping", "Music" ], "b": [ "Music", "Base Jumping" ] }
+{ "a": [ "Base Jumping", "Music" ], "b": [ "Music", "Base Jumping" ] }
+{ "a": [ "Fishing", "Video Games" ], "b": [ "Video Games", "Fishing" ] }
+{ "a": [ "Base Jumping", "Skiing" ], "b": [ "Skiing", "Base Jumping" ] }
+{ "a": [ "Base Jumping", "Skiing" ], "b": [ "Base Jumping", "Skiing" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Skiing", "Bass" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Skiing", "Bass" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Bass", "Skiing" ] }
+{ "a": [ "Fishing", "Running", "Cigars" ], "b": [ "Fishing", "Cigars", "Running" ] }
+{ "a": [ "Bass", "Bass", "Books" ], "b": [ "Movies", "Books", "Bass" ] }
+{ "a": [ "Bass", "Bass", "Books" ], "b": [ "Bass", "Books", "Books" ] }
+{ "a": [ "Cigars", "Skiing" ], "b": [ "Skiing", "Cigars" ] }
+{ "a": [ "Tennis", "Tennis", "Databases", "Squash" ], "b": [ "Cigars", "Databases", "Squash", "Tennis" ] }
+{ "a": [ "Cigars", "Cigars", "Bass", "Books" ], "b": [ "Books", "Cigars", "Bass", "Base Jumping" ] }
+{ "a": [ "Cigars", "Cigars", "Bass", "Books" ], "b": [ "Bass", "Cigars", "Books", "Basketball" ] }
+{ "a": [ "Movies", "Walking" ], "b": [ "Movies", "Walking" ] }
+{ "a": [ "Music", "Coffee" ], "b": [ "Coffee", "Music" ] }
+{ "a": [ "Running", "Coffee", "Fishing" ], "b": [ "Running", "Fishing", "Coffee" ] }
+{ "a": [ "Squash", "Movies", "Coffee" ], "b": [ "Coffee", "Movies", "Squash" ] }
+{ "a": [ "Music", "Tennis", "Base Jumping" ], "b": [ "Music", "Base Jumping", "Tennis" ] }
+{ "a": [ "Movies", "Fishing", "Fishing" ], "b": [ "Tennis", "Fishing", "Movies" ] }
+{ "a": [ "Movies", "Fishing", "Fishing" ], "b": [ "Databases", "Fishing", "Movies" ] }
+{ "a": [ "Movies", "Fishing", "Fishing" ], "b": [ "Coffee", "Movies", "Fishing" ] }
+{ "a": [ "Music", "Base Jumping", "Books" ], "b": [ "Books", "Base Jumping", "Music" ] }
+{ "a": [ "Bass", "Books" ], "b": [ "Bass", "Books" ] }
+{ "a": [ "Bass", "Books" ], "b": [ "Books", "Bass" ] }
+{ "a": [ "Skiing", "Squash", "Skiing", "Fishing" ], "b": [ "Base Jumping", "Fishing", "Skiing", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Puzzles", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Wine", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Skiing", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Bass", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Video Games", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Tennis" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Music", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Tennis" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Puzzles" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Wine", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Computers" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Bass", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Running", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Base Jumping", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Music", "Squash" ] }
+{ "a": [ "Squash", "Squash" ], "b": [ "Cooking", "Squash" ] }
+{ "a": [ "Puzzles", "Squash" ], "b": [ "Squash", "Puzzles" ] }
+{ "a": [ "Computers", "Wine" ], "b": [ "Wine", "Computers" ] }
+{ "a": [ "Computers", "Wine" ], "b": [ "Computers", "Wine" ] }
+{ "a": [ "Computers", "Wine" ], "b": [ "Wine", "Computers" ] }
+{ "a": [ "Walking", "Cooking" ], "b": [ "Walking", "Cooking" ] }
+{ "a": [ "Walking", "Cooking" ], "b": [ "Walking", "Cooking" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Basketball", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Skiing" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Video Games", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Coffee", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Skiing" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Fishing", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Cooking" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Puzzles" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Fishing", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Wine", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Tennis" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Movies" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Base Jumping", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Coffee" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Wine" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Running", "Databases" ] }
+{ "a": [ "Squash", "Databases" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Music", "Fishing", "Music" ], "b": [ "Wine", "Music", "Fishing" ] }
+{ "a": [ "Music", "Fishing", "Music" ], "b": [ "Music", "Fishing", "Computers" ] }
+{ "a": [ "Wine", "Computers" ], "b": [ "Computers", "Wine" ] }
+{ "a": [ "Wine", "Computers" ], "b": [ "Wine", "Computers" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Skiing", "Bass" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Bass", "Skiing" ] }
+{ "a": [ "Coffee", "Coffee", "Cigars" ], "b": [ "Cigars", "Cigars", "Coffee" ] }
+{ "a": [ "Coffee", "Coffee", "Cigars" ], "b": [ "Cigars", "Coffee", "Books" ] }
+{ "a": [ "Coffee", "Coffee", "Cigars" ], "b": [ "Coffee", "Walking", "Cigars" ] }
+{ "a": [ "Coffee", "Coffee", "Cigars" ], "b": [ "Base Jumping", "Coffee", "Cigars" ] }
+{ "a": [ "Movies", "Books" ], "b": [ "Movies", "Books" ] }
+{ "a": [ "Movies", "Books" ], "b": [ "Books", "Movies" ] }
+{ "a": [ "Movies", "Books" ], "b": [ "Movies", "Books" ] }
+{ "a": [ "Wine", "Squash" ], "b": [ "Wine", "Squash" ] }
+{ "a": [ "Coffee", "Tennis" ], "b": [ "Tennis", "Coffee" ] }
+{ "a": [ "Coffee", "Tennis" ], "b": [ "Tennis", "Coffee" ] }
+{ "a": [ "Skiing", "Books" ], "b": [ "Books", "Skiing" ] }
+{ "a": [ "Databases", "Music" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Music" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Databases", "Music" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Video Games", "Cigars" ], "b": [ "Cigars", "Video Games" ] }
+{ "a": [ "Video Games", "Cigars" ], "b": [ "Video Games", "Cigars" ] }
+{ "a": [ "Databases", "Skiing" ], "b": [ "Databases", "Skiing" ] }
+{ "a": [ "Running", "Fishing" ], "b": [ "Running", "Fishing" ] }
+{ "a": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "b": [ "Base Jumping", "Fishing", "Walking", "Computers" ] }
+{ "a": [ "Databases", "Music" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Databases", "Music" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Wine", "Walking", "Books", "Walking" ], "b": [ "Music", "Wine", "Books", "Walking" ] }
+{ "a": [ "Bass", "Bass", "Base Jumping" ], "b": [ "Base Jumping", "Bass", "Cooking" ] }
+{ "a": [ "Bass", "Bass", "Base Jumping" ], "b": [ "Base Jumping", "Databases", "Bass" ] }
+{ "a": [ "Cigars", "Cigars", "Coffee" ], "b": [ "Cigars", "Coffee", "Books" ] }
+{ "a": [ "Cigars", "Cigars", "Coffee" ], "b": [ "Coffee", "Walking", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars", "Coffee" ], "b": [ "Base Jumping", "Coffee", "Cigars" ] }
+{ "a": [ "Base Jumping", "Running" ], "b": [ "Running", "Base Jumping" ] }
+{ "a": [ "Base Jumping", "Running" ], "b": [ "Base Jumping", "Running" ] }
+{ "a": [ "Cooking", "Squash", "Cooking", "Coffee" ], "b": [ "Coffee", "Cigars", "Cooking", "Squash" ] }
+{ "a": [ "Books", "Base Jumping" ], "b": [ "Books", "Base Jumping" ] }
+{ "a": [ "Books", "Base Jumping" ], "b": [ "Books", "Base Jumping" ] }
+{ "a": [ "Cooking", "Running" ], "b": [ "Cooking", "Running" ] }
+{ "a": [ "Video Games", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Video Games", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Cigars", "Video Games" ], "b": [ "Video Games", "Cigars" ] }
+{ "a": [ "Running", "Base Jumping" ], "b": [ "Base Jumping", "Running" ] }
+{ "a": [ "Coffee", "Databases" ], "b": [ "Databases", "Coffee" ] }
+{ "a": [ "Movies", "Books" ], "b": [ "Books", "Movies" ] }
+{ "a": [ "Movies", "Books" ], "b": [ "Movies", "Books" ] }
+{ "a": [ "Databases", "Video Games" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Tennis", "Movies", "Movies" ], "b": [ "Databases", "Movies", "Tennis" ] }
+{ "a": [ "Tennis", "Movies", "Movies" ], "b": [ "Tennis", "Movies", "Bass" ] }
+{ "a": [ "Tennis", "Movies", "Movies" ], "b": [ "Databases", "Movies", "Tennis" ] }
+{ "a": [ "Music", "Base Jumping" ], "b": [ "Music", "Base Jumping" ] }
+{ "a": [ "Bass", "Squash" ], "b": [ "Bass", "Squash" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Cooking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Books" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Wine", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Running" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Computers" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Bass", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Skiing", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Computers", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Wine" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Movies", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Bass" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Music", "Walking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Walking", "Cooking" ] }
+{ "a": [ "Walking", "Walking" ], "b": [ "Cigars", "Walking" ] }
+{ "a": [ "Computers", "Tennis" ], "b": [ "Tennis", "Computers" ] }
+{ "a": [ "Tennis", "Coffee" ], "b": [ "Tennis", "Coffee" ] }
+{ "a": [ "Running", "Basketball", "Computers", "Basketball" ], "b": [ "Computers", "Cooking", "Running", "Basketball" ] }
+{ "a": [ "Puzzles", "Books" ], "b": [ "Puzzles", "Books" ] }
+{ "a": [ "Puzzles", "Books" ], "b": [ "Puzzles", "Books" ] }
+{ "a": [ "Skiing", "Wine" ], "b": [ "Wine", "Skiing" ] }
+{ "a": [ "Squash", "Squash", "Music" ], "b": [ "Video Games", "Squash", "Music" ] }
+{ "a": [ "Squash", "Tennis" ], "b": [ "Squash", "Tennis" ] }
+{ "a": [ "Walking", "Cooking" ], "b": [ "Walking", "Cooking" ] }
+{ "a": [ "Coffee", "Tennis", "Bass" ], "b": [ "Coffee", "Bass", "Tennis" ] }
+{ "a": [ "Music", "Squash" ], "b": [ "Music", "Squash" ] }
+{ "a": [ "Computers", "Fishing" ], "b": [ "Fishing", "Computers" ] }
+{ "a": [ "Computers", "Fishing" ], "b": [ "Computers", "Fishing" ] }
+{ "a": [ "Wine", "Walking" ], "b": [ "Walking", "Wine" ] }
+{ "a": [ "Skiing", "Base Jumping" ], "b": [ "Base Jumping", "Skiing" ] }
+{ "a": [ "Bass", "Books" ], "b": [ "Books", "Bass" ] }
+{ "a": [ "Tennis", "Running", "Tennis" ], "b": [ "Running", "Basketball", "Tennis" ] }
+{ "a": [ "Tennis", "Running", "Tennis" ], "b": [ "Running", "Tennis", "Video Games" ] }
+{ "a": [ "Fishing", "Music" ], "b": [ "Fishing", "Music" ] }
+{ "a": [ "Books", "Tennis" ], "b": [ "Books", "Tennis" ] }
+{ "a": [ "Books", "Tennis" ], "b": [ "Tennis", "Books" ] }
+{ "a": [ "Squash", "Squash", "Video Games" ], "b": [ "Video Games", "Squash", "Music" ] }
+{ "a": [ "Books", "Tennis" ], "b": [ "Tennis", "Books" ] }
+{ "a": [ "Music", "Books", "Books", "Wine" ], "b": [ "Music", "Wine", "Books", "Walking" ] }
+{ "a": [ "Basketball", "Basketball", "Computers" ], "b": [ "Computers", "Basketball", "Squash" ] }
+{ "a": [ "Fishing", "Databases" ], "b": [ "Fishing", "Databases" ] }
+{ "a": [ "Walking", "Computers" ], "b": [ "Computers", "Walking" ] }
+{ "a": [ "Books", "Base Jumping" ], "b": [ "Books", "Base Jumping" ] }
+{ "a": [ "Movies", "Cooking", "Skiing" ], "b": [ "Movies", "Skiing", "Cooking" ] }
+{ "a": [ "Puzzles", "Books" ], "b": [ "Puzzles", "Books" ] }
+{ "a": [ "Wine", "Databases" ], "b": [ "Databases", "Wine" ] }
+{ "a": [ "Fishing", "Databases", "Wine" ], "b": [ "Fishing", "Wine", "Databases" ] }
+{ "a": [ "Fishing", "Databases", "Wine" ], "b": [ "Databases", "Fishing", "Wine" ] }
+{ "a": [ "Coffee", "Movies", "Skiing" ], "b": [ "Coffee", "Movies", "Skiing" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Base Jumping" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Video Games", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Bass" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Skiing", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Walking" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Fishing", "Cigars" ] }
+{ "a": [ "Bass", "Walking" ], "b": [ "Walking", "Bass" ] }
+{ "a": [ "Wine", "Base Jumping", "Running" ], "b": [ "Base Jumping", "Running", "Wine" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Tennis" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Movies" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Base Jumping", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Coffee" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Wine" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Running", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Movies" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Squash", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Base Jumping", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Video Games" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Music", "Databases" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Coffee" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Databases", "Wine" ] }
+{ "a": [ "Databases", "Databases" ], "b": [ "Running", "Databases" ] }
+{ "a": [ "Fishing", "Skiing", "Skiing" ], "b": [ "Databases", "Fishing", "Skiing" ] }
+{ "a": [ "Base Jumping", "Basketball", "Music", "Basketball" ], "b": [ "Music", "Walking", "Basketball", "Base Jumping" ] }
+{ "a": [ "Movies", "Running" ], "b": [ "Movies", "Running" ] }
+{ "a": [ "Wine", "Puzzles" ], "b": [ "Puzzles", "Wine" ] }
+{ "a": [ "Squash", "Cigars" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Computers", "Coffee", "Walking", "Walking" ], "b": [ "Coffee", "Computers", "Walking", "Basketball" ] }
+{ "a": [ "Tennis", "Music", "Running", "Music" ], "b": [ "Tennis", "Music", "Running", "Cooking" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Fishing", "Movies" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Movies", "Running" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Movies", "Skiing" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Movies", "Walking" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Books", "Movies" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Fishing", "Movies" ] }
+{ "a": [ "Movies", "Movies" ], "b": [ "Movies", "Books" ] }
+{ "a": [ "Squash", "Skiing", "Skiing" ], "b": [ "Squash", "Basketball", "Skiing" ] }
+{ "a": [ "Cooking", "Databases", "Databases" ], "b": [ "Databases", "Cooking", "Wine" ] }
+{ "a": [ "Cooking", "Databases", "Databases" ], "b": [ "Books", "Databases", "Cooking" ] }
+{ "a": [ "Running", "Running" ], "b": [ "Running", "Tennis" ] }
+{ "a": [ "Running", "Running" ], "b": [ "Movies", "Running" ] }
+{ "a": [ "Running", "Running" ], "b": [ "Running", "Squash" ] }
+{ "a": [ "Running", "Running" ], "b": [ "Running", "Databases" ] }
+{ "a": [ "Skiing", "Coffee", "Skiing" ], "b": [ "Coffee", "Movies", "Skiing" ] }
+{ "a": [ "Cooking", "Bass" ], "b": [ "Cooking", "Bass" ] }
+{ "a": [ "Cigars", "Cigars", "Video Games", "Wine" ], "b": [ "Tennis", "Wine", "Cigars", "Video Games" ] }
+{ "a": [ "Databases", "Movies", "Tennis" ], "b": [ "Databases", "Movies", "Tennis" ] }
+{ "a": [ "Fishing", "Computers" ], "b": [ "Computers", "Fishing" ] }
+{ "a": [ "Fishing", "Movies" ], "b": [ "Fishing", "Movies" ] }
+{ "a": [ "Base Jumping", "Tennis", "Video Games" ], "b": [ "Video Games", "Base Jumping", "Tennis" ] }
+{ "a": [ "Computers", "Wine" ], "b": [ "Wine", "Computers" ] }
+{ "a": [ "Fishing", "Fishing" ], "b": [ "Computers", "Fishing" ] }
+{ "a": [ "Fishing", "Fishing" ], "b": [ "Fishing", "Movies" ] }
+{ "a": [ "Fishing", "Fishing" ], "b": [ "Fishing", "Music" ] }
+{ "a": [ "Fishing", "Fishing" ], "b": [ "Fishing", "Cigars" ] }
+{ "a": [ "Books", "Bass", "Books", "Books" ], "b": [ "Books", "Books", "Bass", "Cooking" ] }
+{ "a": [ "Skiing", "Bass" ], "b": [ "Bass", "Skiing" ] }
+{ "a": [ "Music", "Databases" ], "b": [ "Databases", "Music" ] }
+{ "a": [ "Fishing", "Wine", "Databases" ], "b": [ "Databases", "Fishing", "Wine" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Bass" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Squash", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Skiing", "Cigars" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Cigars", "Walking" ] }
+{ "a": [ "Cigars", "Cigars" ], "b": [ "Fishing", "Cigars" ] }
+{ "a": [ "Running", "Wine", "Running" ], "b": [ "Base Jumping", "Running", "Wine" ] }
+{ "a": [ "Books", "Movies" ], "b": [ "Movies", "Books" ] }
+{ "a": [ "Wine", "Wine", "Fishing" ], "b": [ "Databases", "Fishing", "Wine" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ulist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ulist-jaccard.adm
new file mode 100644
index 0000000..55af5a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-ulist-jaccard.adm
@@ -0,0 +1,267 @@
+{ "a": {{ "Bass", "Tennis", "Bass", "Cooking" }}, "b": {{ "Bass", "Cooking", "Running", "Tennis" }} }
+{ "a": {{ "Bass", "Wine" }}, "b": {{ "Bass", "Wine" }} }
+{ "a": {{ "Music", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Music", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Music", "Databases" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Music", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Wine", "Walking" }}, "b": {{ "Wine", "Walking" }} }
+{ "a": {{ "Wine", "Walking" }}, "b": {{ "Walking", "Wine" }} }
+{ "a": {{ "Base Jumping", "Cigars", "Movies" }}, "b": {{ "Base Jumping", "Cigars", "Movies" }} }
+{ "a": {{ "Fishing", "Running", "Tennis", "Running" }}, "b": {{ "Tennis", "Coffee", "Running", "Fishing" }} }
+{ "a": {{ "Skiing", "Walking" }}, "b": {{ "Skiing", "Walking" }} }
+{ "a": {{ "Base Jumping", "Music" }}, "b": {{ "Music", "Base Jumping" }} }
+{ "a": {{ "Base Jumping", "Music" }}, "b": {{ "Music", "Base Jumping" }} }
+{ "a": {{ "Fishing", "Video Games" }}, "b": {{ "Video Games", "Fishing" }} }
+{ "a": {{ "Base Jumping", "Skiing" }}, "b": {{ "Skiing", "Base Jumping" }} }
+{ "a": {{ "Base Jumping", "Skiing" }}, "b": {{ "Base Jumping", "Skiing" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Skiing", "Bass" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Skiing", "Bass" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Bass", "Skiing" }} }
+{ "a": {{ "Fishing", "Running", "Cigars" }}, "b": {{ "Fishing", "Cigars", "Running" }} }
+{ "a": {{ "Bass", "Bass", "Books" }}, "b": {{ "Movies", "Books", "Bass" }} }
+{ "a": {{ "Bass", "Bass", "Books" }}, "b": {{ "Bass", "Books", "Books" }} }
+{ "a": {{ "Cigars", "Skiing" }}, "b": {{ "Skiing", "Cigars" }} }
+{ "a": {{ "Tennis", "Tennis", "Databases", "Squash" }}, "b": {{ "Cigars", "Databases", "Squash", "Tennis" }} }
+{ "a": {{ "Cigars", "Cigars", "Bass", "Books" }}, "b": {{ "Books", "Cigars", "Bass", "Base Jumping" }} }
+{ "a": {{ "Cigars", "Cigars", "Bass", "Books" }}, "b": {{ "Bass", "Cigars", "Books", "Basketball" }} }
+{ "a": {{ "Movies", "Walking" }}, "b": {{ "Movies", "Walking" }} }
+{ "a": {{ "Music", "Coffee" }}, "b": {{ "Coffee", "Music" }} }
+{ "a": {{ "Running", "Coffee", "Fishing" }}, "b": {{ "Running", "Fishing", "Coffee" }} }
+{ "a": {{ "Squash", "Movies", "Coffee" }}, "b": {{ "Coffee", "Movies", "Squash" }} }
+{ "a": {{ "Music", "Tennis", "Base Jumping" }}, "b": {{ "Music", "Base Jumping", "Tennis" }} }
+{ "a": {{ "Movies", "Fishing", "Fishing" }}, "b": {{ "Tennis", "Fishing", "Movies" }} }
+{ "a": {{ "Movies", "Fishing", "Fishing" }}, "b": {{ "Databases", "Fishing", "Movies" }} }
+{ "a": {{ "Movies", "Fishing", "Fishing" }}, "b": {{ "Coffee", "Movies", "Fishing" }} }
+{ "a": {{ "Music", "Base Jumping", "Books" }}, "b": {{ "Books", "Base Jumping", "Music" }} }
+{ "a": {{ "Bass", "Books" }}, "b": {{ "Bass", "Books" }} }
+{ "a": {{ "Bass", "Books" }}, "b": {{ "Books", "Bass" }} }
+{ "a": {{ "Skiing", "Squash", "Skiing", "Fishing" }}, "b": {{ "Base Jumping", "Fishing", "Skiing", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Puzzles", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Wine", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Skiing", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Bass", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Video Games", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Tennis" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Music", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Tennis" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Puzzles" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Wine", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Computers" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Bass", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Running", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Base Jumping", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Music", "Squash" }} }
+{ "a": {{ "Squash", "Squash" }}, "b": {{ "Cooking", "Squash" }} }
+{ "a": {{ "Puzzles", "Squash" }}, "b": {{ "Squash", "Puzzles" }} }
+{ "a": {{ "Computers", "Wine" }}, "b": {{ "Wine", "Computers" }} }
+{ "a": {{ "Computers", "Wine" }}, "b": {{ "Computers", "Wine" }} }
+{ "a": {{ "Computers", "Wine" }}, "b": {{ "Wine", "Computers" }} }
+{ "a": {{ "Walking", "Cooking" }}, "b": {{ "Walking", "Cooking" }} }
+{ "a": {{ "Walking", "Cooking" }}, "b": {{ "Walking", "Cooking" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Basketball", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Skiing" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Video Games", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Coffee", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Skiing" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Fishing", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Cooking" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Puzzles" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Fishing", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Wine", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Tennis" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Movies" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Base Jumping", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Coffee" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Wine" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Running", "Databases" }} }
+{ "a": {{ "Squash", "Databases" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Music", "Fishing", "Music" }}, "b": {{ "Wine", "Music", "Fishing" }} }
+{ "a": {{ "Music", "Fishing", "Music" }}, "b": {{ "Music", "Fishing", "Computers" }} }
+{ "a": {{ "Wine", "Computers" }}, "b": {{ "Computers", "Wine" }} }
+{ "a": {{ "Wine", "Computers" }}, "b": {{ "Wine", "Computers" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Skiing", "Bass" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Bass", "Skiing" }} }
+{ "a": {{ "Coffee", "Coffee", "Cigars" }}, "b": {{ "Cigars", "Cigars", "Coffee" }} }
+{ "a": {{ "Coffee", "Coffee", "Cigars" }}, "b": {{ "Cigars", "Coffee", "Books" }} }
+{ "a": {{ "Coffee", "Coffee", "Cigars" }}, "b": {{ "Coffee", "Walking", "Cigars" }} }
+{ "a": {{ "Coffee", "Coffee", "Cigars" }}, "b": {{ "Base Jumping", "Coffee", "Cigars" }} }
+{ "a": {{ "Movies", "Books" }}, "b": {{ "Movies", "Books" }} }
+{ "a": {{ "Movies", "Books" }}, "b": {{ "Books", "Movies" }} }
+{ "a": {{ "Movies", "Books" }}, "b": {{ "Movies", "Books" }} }
+{ "a": {{ "Wine", "Squash" }}, "b": {{ "Wine", "Squash" }} }
+{ "a": {{ "Coffee", "Tennis" }}, "b": {{ "Tennis", "Coffee" }} }
+{ "a": {{ "Coffee", "Tennis" }}, "b": {{ "Tennis", "Coffee" }} }
+{ "a": {{ "Skiing", "Books" }}, "b": {{ "Books", "Skiing" }} }
+{ "a": {{ "Databases", "Music" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Music" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Databases", "Music" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Video Games", "Cigars" }}, "b": {{ "Cigars", "Video Games" }} }
+{ "a": {{ "Video Games", "Cigars" }}, "b": {{ "Video Games", "Cigars" }} }
+{ "a": {{ "Databases", "Skiing" }}, "b": {{ "Databases", "Skiing" }} }
+{ "a": {{ "Running", "Fishing" }}, "b": {{ "Running", "Fishing" }} }
+{ "a": {{ "Base Jumping", "Base Jumping", "Walking", "Computers" }}, "b": {{ "Base Jumping", "Fishing", "Walking", "Computers" }} }
+{ "a": {{ "Databases", "Music" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Databases", "Music" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Wine", "Walking", "Books", "Walking" }}, "b": {{ "Music", "Wine", "Books", "Walking" }} }
+{ "a": {{ "Bass", "Bass", "Base Jumping" }}, "b": {{ "Base Jumping", "Bass", "Cooking" }} }
+{ "a": {{ "Bass", "Bass", "Base Jumping" }}, "b": {{ "Base Jumping", "Databases", "Bass" }} }
+{ "a": {{ "Cigars", "Cigars", "Coffee" }}, "b": {{ "Cigars", "Coffee", "Books" }} }
+{ "a": {{ "Cigars", "Cigars", "Coffee" }}, "b": {{ "Coffee", "Walking", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars", "Coffee" }}, "b": {{ "Base Jumping", "Coffee", "Cigars" }} }
+{ "a": {{ "Base Jumping", "Running" }}, "b": {{ "Running", "Base Jumping" }} }
+{ "a": {{ "Base Jumping", "Running" }}, "b": {{ "Base Jumping", "Running" }} }
+{ "a": {{ "Cooking", "Squash", "Cooking", "Coffee" }}, "b": {{ "Coffee", "Cigars", "Cooking", "Squash" }} }
+{ "a": {{ "Books", "Base Jumping" }}, "b": {{ "Books", "Base Jumping" }} }
+{ "a": {{ "Books", "Base Jumping" }}, "b": {{ "Books", "Base Jumping" }} }
+{ "a": {{ "Cooking", "Running" }}, "b": {{ "Cooking", "Running" }} }
+{ "a": {{ "Video Games", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Video Games", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Cigars", "Video Games" }}, "b": {{ "Video Games", "Cigars" }} }
+{ "a": {{ "Running", "Base Jumping" }}, "b": {{ "Base Jumping", "Running" }} }
+{ "a": {{ "Coffee", "Databases" }}, "b": {{ "Databases", "Coffee" }} }
+{ "a": {{ "Movies", "Books" }}, "b": {{ "Books", "Movies" }} }
+{ "a": {{ "Movies", "Books" }}, "b": {{ "Movies", "Books" }} }
+{ "a": {{ "Databases", "Video Games" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Tennis", "Movies", "Movies" }}, "b": {{ "Databases", "Movies", "Tennis" }} }
+{ "a": {{ "Tennis", "Movies", "Movies" }}, "b": {{ "Tennis", "Movies", "Bass" }} }
+{ "a": {{ "Tennis", "Movies", "Movies" }}, "b": {{ "Databases", "Movies", "Tennis" }} }
+{ "a": {{ "Music", "Base Jumping" }}, "b": {{ "Music", "Base Jumping" }} }
+{ "a": {{ "Bass", "Squash" }}, "b": {{ "Bass", "Squash" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Cooking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Books" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Wine", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Running" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Computers" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Bass", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Skiing", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Computers", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Wine" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Movies", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Bass" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Music", "Walking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Walking", "Cooking" }} }
+{ "a": {{ "Walking", "Walking" }}, "b": {{ "Cigars", "Walking" }} }
+{ "a": {{ "Computers", "Tennis" }}, "b": {{ "Tennis", "Computers" }} }
+{ "a": {{ "Tennis", "Coffee" }}, "b": {{ "Tennis", "Coffee" }} }
+{ "a": {{ "Running", "Basketball", "Computers", "Basketball" }}, "b": {{ "Computers", "Cooking", "Running", "Basketball" }} }
+{ "a": {{ "Puzzles", "Books" }}, "b": {{ "Puzzles", "Books" }} }
+{ "a": {{ "Puzzles", "Books" }}, "b": {{ "Puzzles", "Books" }} }
+{ "a": {{ "Skiing", "Wine" }}, "b": {{ "Wine", "Skiing" }} }
+{ "a": {{ "Squash", "Squash", "Music" }}, "b": {{ "Video Games", "Squash", "Music" }} }
+{ "a": {{ "Squash", "Tennis" }}, "b": {{ "Squash", "Tennis" }} }
+{ "a": {{ "Walking", "Cooking" }}, "b": {{ "Walking", "Cooking" }} }
+{ "a": {{ "Coffee", "Tennis", "Bass" }}, "b": {{ "Coffee", "Bass", "Tennis" }} }
+{ "a": {{ "Music", "Squash" }}, "b": {{ "Music", "Squash" }} }
+{ "a": {{ "Computers", "Fishing" }}, "b": {{ "Fishing", "Computers" }} }
+{ "a": {{ "Computers", "Fishing" }}, "b": {{ "Computers", "Fishing" }} }
+{ "a": {{ "Wine", "Walking" }}, "b": {{ "Walking", "Wine" }} }
+{ "a": {{ "Skiing", "Base Jumping" }}, "b": {{ "Base Jumping", "Skiing" }} }
+{ "a": {{ "Bass", "Books" }}, "b": {{ "Books", "Bass" }} }
+{ "a": {{ "Tennis", "Running", "Tennis" }}, "b": {{ "Running", "Basketball", "Tennis" }} }
+{ "a": {{ "Tennis", "Running", "Tennis" }}, "b": {{ "Running", "Tennis", "Video Games" }} }
+{ "a": {{ "Fishing", "Music" }}, "b": {{ "Fishing", "Music" }} }
+{ "a": {{ "Books", "Tennis" }}, "b": {{ "Books", "Tennis" }} }
+{ "a": {{ "Books", "Tennis" }}, "b": {{ "Tennis", "Books" }} }
+{ "a": {{ "Squash", "Squash", "Video Games" }}, "b": {{ "Video Games", "Squash", "Music" }} }
+{ "a": {{ "Books", "Tennis" }}, "b": {{ "Tennis", "Books" }} }
+{ "a": {{ "Music", "Books", "Books", "Wine" }}, "b": {{ "Music", "Wine", "Books", "Walking" }} }
+{ "a": {{ "Basketball", "Basketball", "Computers" }}, "b": {{ "Computers", "Basketball", "Squash" }} }
+{ "a": {{ "Fishing", "Databases" }}, "b": {{ "Fishing", "Databases" }} }
+{ "a": {{ "Walking", "Computers" }}, "b": {{ "Computers", "Walking" }} }
+{ "a": {{ "Books", "Base Jumping" }}, "b": {{ "Books", "Base Jumping" }} }
+{ "a": {{ "Movies", "Cooking", "Skiing" }}, "b": {{ "Movies", "Skiing", "Cooking" }} }
+{ "a": {{ "Puzzles", "Books" }}, "b": {{ "Puzzles", "Books" }} }
+{ "a": {{ "Wine", "Databases" }}, "b": {{ "Databases", "Wine" }} }
+{ "a": {{ "Fishing", "Databases", "Wine" }}, "b": {{ "Fishing", "Wine", "Databases" }} }
+{ "a": {{ "Fishing", "Databases", "Wine" }}, "b": {{ "Databases", "Fishing", "Wine" }} }
+{ "a": {{ "Coffee", "Movies", "Skiing" }}, "b": {{ "Coffee", "Movies", "Skiing" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Base Jumping" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Video Games", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Bass" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Skiing", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Walking" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Fishing", "Cigars" }} }
+{ "a": {{ "Bass", "Walking" }}, "b": {{ "Walking", "Bass" }} }
+{ "a": {{ "Wine", "Base Jumping", "Running" }}, "b": {{ "Base Jumping", "Running", "Wine" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Tennis" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Movies" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Base Jumping", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Coffee" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Wine" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Running", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Movies" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Squash", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Base Jumping", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Video Games" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Music", "Databases" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Coffee" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Databases", "Wine" }} }
+{ "a": {{ "Databases", "Databases" }}, "b": {{ "Running", "Databases" }} }
+{ "a": {{ "Fishing", "Skiing", "Skiing" }}, "b": {{ "Databases", "Fishing", "Skiing" }} }
+{ "a": {{ "Base Jumping", "Basketball", "Music", "Basketball" }}, "b": {{ "Music", "Walking", "Basketball", "Base Jumping" }} }
+{ "a": {{ "Movies", "Running" }}, "b": {{ "Movies", "Running" }} }
+{ "a": {{ "Wine", "Puzzles" }}, "b": {{ "Puzzles", "Wine" }} }
+{ "a": {{ "Squash", "Cigars" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Computers", "Coffee", "Walking", "Walking" }}, "b": {{ "Coffee", "Computers", "Walking", "Basketball" }} }
+{ "a": {{ "Tennis", "Music", "Running", "Music" }}, "b": {{ "Tennis", "Music", "Running", "Cooking" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Fishing", "Movies" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Movies", "Running" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Movies", "Skiing" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Movies", "Walking" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Books", "Movies" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Fishing", "Movies" }} }
+{ "a": {{ "Movies", "Movies" }}, "b": {{ "Movies", "Books" }} }
+{ "a": {{ "Squash", "Skiing", "Skiing" }}, "b": {{ "Squash", "Basketball", "Skiing" }} }
+{ "a": {{ "Cooking", "Databases", "Databases" }}, "b": {{ "Databases", "Cooking", "Wine" }} }
+{ "a": {{ "Cooking", "Databases", "Databases" }}, "b": {{ "Books", "Databases", "Cooking" }} }
+{ "a": {{ "Running", "Running" }}, "b": {{ "Running", "Tennis" }} }
+{ "a": {{ "Running", "Running" }}, "b": {{ "Movies", "Running" }} }
+{ "a": {{ "Running", "Running" }}, "b": {{ "Running", "Squash" }} }
+{ "a": {{ "Running", "Running" }}, "b": {{ "Running", "Databases" }} }
+{ "a": {{ "Skiing", "Coffee", "Skiing" }}, "b": {{ "Coffee", "Movies", "Skiing" }} }
+{ "a": {{ "Cooking", "Bass" }}, "b": {{ "Cooking", "Bass" }} }
+{ "a": {{ "Cigars", "Cigars", "Video Games", "Wine" }}, "b": {{ "Tennis", "Wine", "Cigars", "Video Games" }} }
+{ "a": {{ "Databases", "Movies", "Tennis" }}, "b": {{ "Databases", "Movies", "Tennis" }} }
+{ "a": {{ "Fishing", "Computers" }}, "b": {{ "Computers", "Fishing" }} }
+{ "a": {{ "Fishing", "Movies" }}, "b": {{ "Fishing", "Movies" }} }
+{ "a": {{ "Base Jumping", "Tennis", "Video Games" }}, "b": {{ "Video Games", "Base Jumping", "Tennis" }} }
+{ "a": {{ "Computers", "Wine" }}, "b": {{ "Wine", "Computers" }} }
+{ "a": {{ "Fishing", "Fishing" }}, "b": {{ "Computers", "Fishing" }} }
+{ "a": {{ "Fishing", "Fishing" }}, "b": {{ "Fishing", "Movies" }} }
+{ "a": {{ "Fishing", "Fishing" }}, "b": {{ "Fishing", "Music" }} }
+{ "a": {{ "Fishing", "Fishing" }}, "b": {{ "Fishing", "Cigars" }} }
+{ "a": {{ "Books", "Bass", "Books", "Books" }}, "b": {{ "Books", "Books", "Bass", "Cooking" }} }
+{ "a": {{ "Skiing", "Bass" }}, "b": {{ "Bass", "Skiing" }} }
+{ "a": {{ "Music", "Databases" }}, "b": {{ "Databases", "Music" }} }
+{ "a": {{ "Fishing", "Wine", "Databases" }}, "b": {{ "Databases", "Fishing", "Wine" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Bass" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Squash", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Skiing", "Cigars" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Cigars", "Walking" }} }
+{ "a": {{ "Cigars", "Cigars" }}, "b": {{ "Fishing", "Cigars" }} }
+{ "a": {{ "Running", "Wine", "Running" }}, "b": {{ "Base Jumping", "Running", "Wine" }} }
+{ "a": {{ "Books", "Movies" }}, "b": {{ "Movies", "Books" }} }
+{ "a": {{ "Wine", "Wine", "Fishing" }}, "b": {{ "Databases", "Fishing", "Wine" }} }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-word-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-word-jaccard.adm
new file mode 100644
index 0000000..9793e0b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-join/inverted-index-word-jaccard.adm
@@ -0,0 +1,5 @@
+{ "arec": "Active Database Systems.", "brec": "Active Database Systems" }
+{ "arec": "Specification and Execution of Transactional Workflows.", "brec": "Specification and Execution of Transactional Workflows" }
+{ "arec": "Integrated Office Systems.", "brec": "Integrated Office Systems" }
+{ "arec": "Integrated Office Systems.", "brec": "Integrated Office Systems" }
+{ "arec": "A Shared View of Sharing  The Treaty of Orlando.", "brec": "A Shared View of Sharing  The Treaty of Orlando" }
diff --git a/asterix-app/src/test/resources/runtimets/results/index/cust-index-age-nullable.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/cust-index-age-nullable.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/cust-index-age-nullable.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/cust-index-age-nullable.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-contains.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-contains.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-contains.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance-panic.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance-panic.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance-panic.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-edit-distance.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-jaccard.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ngram-jaccard.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance-panic.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance-panic.adm
new file mode 100644
index 0000000..9e33b16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance-panic.adm
@@ -0,0 +1,854 @@
+{ "cid": 1, "name": "Trudie Minick", "age": 75, "address": { "number": 6740, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing", "Squash" ], "children": [ { "name": "Arie Minick", "age": 56 }, { "name": "Alline Minick", "age": 57 }, { "name": "Petronila Minick", "age": 56 } ] }
+{ "cid": 2, "name": "Elin Debell", "age": 82, "address": { "number": 5649, "street": "Hill St.", "city": "Portland" }, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Elvina Debell", "age": null }, { "name": "Renaldo Debell", "age": 51 }, { "name": "Divina Debell", "age": 57 } ] }
+{ "cid": 3, "name": "Phung Wheetley", "age": 12, "address": { "number": 5549, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Raelene Wheetley", "age": null }, { "name": "Dudley Wheetley", "age": null } ] }
+{ "cid": 4, "name": "Bernita Gungor", "age": 87, "address": { "number": 1208, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Valencia Gungor", "age": 72 }, { "name": "Evangeline Gungor", "age": 76 }, { "name": "Odell Gungor", "age": null }, { "name": "Denny Gungor", "age": null } ] }
+{ "cid": 5, "name": "Heide Naifeh", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Deirdre Naifeh", "age": null }, { "name": "Jacquelyne Naifeh", "age": 39 } ] }
+{ "cid": 6, "name": "Cris Kager", "age": 70, "address": { "number": 8402, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Carmelo Kager", "age": 34 }, { "name": "Faustina Kager", "age": null } ] }
+{ "cid": 7, "name": "Karie Kaehler", "age": 59, "address": { "number": 9875, "street": "View St.", "city": "San Jose" }, "interests": [ "Computers", "Skiing", "Basketball", "Movies" ], "children": [ { "name": "Spring Kaehler", "age": 17 } ] }
+{ "cid": 8, "name": "Audria Haylett", "age": 44, "address": { "number": 4872, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cooking", "Fishing", "Video Games" ], "children": [ { "name": "Lacie Haylett", "age": 19 } ] }
+{ "cid": 9, "name": "Dreama Nuccio", "age": 55, "address": { "number": 95, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Ricardo Nuccio", "age": 28 }, { "name": "See Nuccio", "age": 34 } ] }
+{ "cid": 10, "name": "Trent Liedy", "age": 51, "address": { "number": 1758, "street": "Oak St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 12, "name": "Laurinda Raimann", "age": null, "address": null, "interests": [ "Basketball", "Coffee" ], "children": [ { "name": "Lulu Raimann", "age": null }, { "name": "Refugia Raimann", "age": 19 }, { "name": "Jimmie Raimann", "age": 10 }, { "name": "Cindy Raimann", "age": null } ] }
+{ "cid": 13, "name": "Nicol Kolmer", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Erika Kolmer", "age": 40 }, { "name": "Justin Kolmer", "age": null }, { "name": "Dorathy Kolmer", "age": null }, { "name": "Anastacia Kolmer", "age": 27 } ] }
+{ "cid": 14, "name": "Chance Nicoson", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Willette Nicoson", "age": 39 }, { "name": "Glennis Nicoson", "age": null }, { "name": "Philip Nicoson", "age": null }, { "name": "Cody Nicoson", "age": 26 } ] }
+{ "cid": 15, "name": "Berry Faubel", "age": 55, "address": { "number": 2806, "street": "Oak St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Tiffiny Faubel", "age": 12 }, { "name": "Hilaria Faubel", "age": 19 }, { "name": "Wesley Faubel", "age": 37 }, { "name": "Wei Faubel", "age": 28 } ] }
+{ "cid": 16, "name": "Felisa Auletta", "age": 55, "address": { "number": 7737, "street": "View St.", "city": "San Jose" }, "interests": [ "Skiing", "Coffee", "Wine" ], "children": [ { "name": "Rosalia Auletta", "age": 36 } ] }
+{ "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 18, "name": "Dewayne Ardan", "age": 32, "address": { "number": 8229, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Wine", "Walking", "Bass" ], "children": [ { "name": "Wen Ardan", "age": null }, { "name": "Sachiko Ardan", "age": 11 }, { "name": "Francis Ardan", "age": 20 } ] }
+{ "cid": 20, "name": "Annice Fulwider", "age": 59, "address": { "number": 4257, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Arica Fulwider", "age": 47 }, { "name": "Charlotte Fulwider", "age": 16 }, { "name": "Robbi Fulwider", "age": 29 } ] }
+{ "cid": 21, "name": "Gidget Galamay", "age": 34, "address": { "number": 2854, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Brunilda Galamay", "age": null }, { "name": "Bethel Galamay", "age": null }, { "name": "Devon Galamay", "age": 17 } ] }
+{ "cid": 22, "name": "Sarita Burrer", "age": null, "address": null, "interests": [ "Cigars", "Computers" ], "children": [  ] }
+{ "cid": 23, "name": "Micheal Konen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Myong Konen", "age": 26 }, { "name": "Celinda Konen", "age": 33 }, { "name": "Tammy Konen", "age": 53 }, { "name": "Chester Konen", "age": null } ] }
+{ "cid": 24, "name": "Hosea Wilburn", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 25, "name": "Goldie Vanhandel", "age": 37, "address": { "number": 6568, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Fishing", "Cigars" ], "children": [  ] }
+{ "cid": 26, "name": "Jone Okuna", "age": 78, "address": { "number": 6006, "street": "7th St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Franchesca Okuna", "age": null }, { "name": "Fred Okuna", "age": 17 }, { "name": "Marcellus Okuna", "age": null } ] }
+{ "cid": 27, "name": "Hollie Hyun", "age": null, "address": null, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Morton Hyun", "age": null }, { "name": "Farrah Hyun", "age": 40 }, { "name": "Ali Hyun", "age": null } ] }
+{ "cid": 28, "name": "Ariana Gillert", "age": 54, "address": { "number": 7331, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Inge Gillert", "age": null }, { "name": "Jeraldine Gillert", "age": 13 } ] }
+{ "cid": 29, "name": "Ruthanne Tavana", "age": null, "address": null, "interests": [ "Movies" ], "children": [  ] }
+{ "cid": 30, "name": "Deedee Centner", "age": null, "address": null, "interests": [ "Skiing", "Wine", "Databases", "Movies" ], "children": [ { "name": "Lorilee Centner", "age": 30 }, { "name": "Thad Centner", "age": null } ] }
+{ "cid": 31, "name": "Venus Toboz", "age": 44, "address": { "number": 9465, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Ashlie Toboz", "age": null } ] }
+{ "cid": 32, "name": "Tia Berkley", "age": 30, "address": { "number": 4507, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Music" ], "children": [ { "name": "Carmon Berkley", "age": null }, { "name": "Kristina Berkley", "age": null }, { "name": "Cristi Berkley", "age": 19 } ] }
+{ "cid": 33, "name": "Rayford Velmontes", "age": null, "address": null, "interests": [ "Fishing", "Video Games" ], "children": [  ] }
+{ "cid": 34, "name": "Sam Tannahill", "age": null, "address": null, "interests": [ "Books" ], "children": [  ] }
+{ "cid": 36, "name": "Neoma Preist", "age": 69, "address": { "number": 4830, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Databases", "Computers", "Coffee" ], "children": [ { "name": "Shery Preist", "age": null }, { "name": "Kelvin Preist", "age": 43 } ] }
+{ "cid": 37, "name": "Eliana Vient", "age": 89, "address": { "number": 4882, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Dario Vient", "age": 43 } ] }
+{ "cid": 38, "name": "Lawanna Abadi", "age": 35, "address": { "number": 6942, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Arthur Abadi", "age": 10 } ] }
+{ "cid": 39, "name": "Brock Froncillo", "age": 72, "address": { "number": 4645, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Cole Froncillo", "age": null }, { "name": "Ivana Froncillo", "age": null }, { "name": "Hugh Froncillo", "age": 23 } ] }
+{ "cid": 40, "name": "Fidelia Connie", "age": 81, "address": { "number": 2298, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Base Jumping", "Walking", "Skiing" ], "children": [ { "name": "Elfreda Connie", "age": 43 }, { "name": "Josephine Connie", "age": 30 }, { "name": "Lucas Connie", "age": null } ] }
+{ "cid": 41, "name": "Kevin Giottonini", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Victor Giottonini", "age": 37 }, { "name": "Alverta Giottonini", "age": null } ] }
+{ "cid": 42, "name": "Asley Simco", "age": 38, "address": { "number": 3322, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Fishing", "Running", "Cigars" ], "children": [ { "name": "Micheal Simco", "age": null }, { "name": "Lawerence Simco", "age": null } ] }
+{ "cid": 44, "name": "Agustin Clubs", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Maxwell Clubs", "age": 31 }, { "name": "Rayna Clubs", "age": null }, { "name": "Darwin Clubs", "age": null } ] }
+{ "cid": 46, "name": "Columbus Huntington", "age": 22, "address": { "number": 3809, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies" ], "children": [ { "name": "Dana Huntington", "age": 10 }, { "name": "Rosa Huntington", "age": null } ] }
+{ "cid": 48, "name": "Delia Salveson", "age": 44, "address": { "number": 5596, "street": "7th St.", "city": "Portland" }, "interests": [ "Cigars", "Running", "Walking", "Running" ], "children": [ { "name": "Logan Salveson", "age": 21 }, { "name": "Temple Salveson", "age": 17 }, { "name": "Kimi Salveson", "age": null }, { "name": "Jacob Salveson", "age": 20 } ] }
+{ "cid": 49, "name": "Asa Schwing", "age": 70, "address": { "number": 2261, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Tennis" ], "children": [ { "name": "Joy Schwing", "age": 15 } ] }
+{ "cid": 50, "name": "Lise Gorelli", "age": null, "address": null, "interests": [ "Books", "Wine", "Skiing", "Computers" ], "children": [ { "name": "Darleen Gorelli", "age": null }, { "name": "Latia Gorelli", "age": null }, { "name": "Page Gorelli", "age": null }, { "name": "Columbus Gorelli", "age": null } ] }
+{ "cid": 51, "name": "Simonne Cape", "age": null, "address": null, "interests": [ "Bass", "Bass", "Books" ], "children": [ { "name": "Leland Cape", "age": null }, { "name": "Gearldine Cape", "age": null } ] }
+{ "cid": 52, "name": "Janna Tish", "age": 12, "address": { "number": 2598, "street": "Washington St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Mackenzie Tish", "age": null }, { "name": "Ettie Tish", "age": null }, { "name": "Hortencia Tish", "age": null }, { "name": "Paul Tish", "age": null } ] }
+{ "cid": 53, "name": "Ricardo Greiwe", "age": 24, "address": { "number": 8983, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 54, "name": "Haywood Vasiloff", "age": 63, "address": { "number": 8780, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Celsa Vasiloff", "age": 40 }, { "name": "Shawana Vasiloff", "age": 43 }, { "name": "Joel Vasiloff", "age": 42 }, { "name": "Timmy Vasiloff", "age": 33 } ] }
+{ "cid": 55, "name": "Terrence Bryant", "age": 12, "address": { "number": 3188, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine", "Cooking" ], "children": [ { "name": "Dayna Bryant", "age": null } ] }
+{ "cid": 56, "name": "Andria Killelea", "age": null, "address": null, "interests": [ "Cigars", "Skiing" ], "children": [  ] }
+{ "cid": 57, "name": "Celestine Mac", "age": null, "address": null, "interests": [ "Wine", "Computers", "Books" ], "children": [ { "name": "Kathyrn Mac", "age": 44 } ] }
+{ "cid": 58, "name": "Rosemarie Mattei", "age": 80, "address": { "number": 1390, "street": "Park St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Sonya Mattei", "age": 52 }, { "name": "Elenor Mattei", "age": null } ] }
+{ "cid": 59, "name": "Rea Villicana", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 61, "name": "Linsey Mose", "age": 17, "address": { "number": 9198, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [ { "name": "Tilda Mose", "age": null }, { "name": "Lillie Mose", "age": null }, { "name": "Robyn Mose", "age": null } ] }
+{ "cid": 62, "name": "Kiley Machnik", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 64, "name": "Victor Susor", "age": 32, "address": { "number": 1690, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Computers" ], "children": [  ] }
+{ "cid": 66, "name": "Lenny Latson", "age": null, "address": null, "interests": [ "Music", "Video Games" ], "children": [  ] }
+{ "cid": 67, "name": "Tobie Mattan", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 68, "name": "Chery Basini", "age": null, "address": null, "interests": [ "Video Games" ], "children": [  ] }
+{ "cid": 69, "name": "Many Yeargain", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brande Yeargain", "age": null }, { "name": "Tawna Yeargain", "age": null }, { "name": "Doris Yeargain", "age": null }, { "name": "Valeria Yeargain", "age": 51 } ] }
+{ "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }
+{ "cid": 71, "name": "Alva Sieger", "age": null, "address": null, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Renetta Sieger", "age": null }, { "name": "Shiloh Sieger", "age": 57 }, { "name": "Lavina Sieger", "age": null }, { "name": "Larraine Sieger", "age": null } ] }
+{ "cid": 73, "name": "Kelsey Flever", "age": 20, "address": { "number": 3555, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Puzzles", "Video Games" ], "children": [ { "name": "Isis Flever", "age": null }, { "name": "Gonzalo Flever", "age": null } ] }
+{ "cid": 74, "name": "Lonnie Ercolani", "age": 79, "address": { "number": 2655, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Music", "Coffee" ], "children": [ { "name": "Cassi Ercolani", "age": null } ] }
+{ "cid": 76, "name": "Opal Blewett", "age": null, "address": null, "interests": [ "Running", "Coffee", "Fishing" ], "children": [ { "name": "Violette Blewett", "age": null } ] }
+{ "cid": 77, "name": "Chantal Parriera", "age": 78, "address": { "number": 5967, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Squash", "Movies", "Coffee" ], "children": [  ] }
+{ "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }
+{ "cid": 79, "name": "Alyce Schoenle", "age": 57, "address": { "number": 1345, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Stewart Schoenle", "age": 16 }, { "name": "Bruce Schoenle", "age": 44 } ] }
+{ "cid": 81, "name": "Lavonda Manford", "age": 87, "address": { "number": 2423, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 82, "name": "Gloria Junkins", "age": null, "address": null, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 83, "name": "Filiberto Couillard", "age": null, "address": null, "interests": [ "Cooking", "Books" ], "children": [ { "name": "Diane Couillard", "age": 19 }, { "name": "Asa Couillard", "age": 23 }, { "name": "Zaida Couillard", "age": 57 }, { "name": "Shavonne Couillard", "age": null } ] }
+{ "cid": 84, "name": "Huong Kachel", "age": null, "address": null, "interests": [ "Music", "Tennis", "Base Jumping" ], "children": [ { "name": "Katlyn Kachel", "age": 40 }, { "name": "Sherman Kachel", "age": null }, { "name": "Susana Kachel", "age": 32 } ] }
+{ "cid": 85, "name": "Fatimah Steltenpohl", "age": 25, "address": { "number": 6175, "street": "Park St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Genoveva Steltenpohl", "age": 14 } ] }
+{ "cid": 86, "name": "Sofia Mongiovi", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Rosamaria Mongiovi", "age": 25 } ] }
+{ "cid": 87, "name": "Torie Horuath", "age": 21, "address": { "number": 2713, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Puzzles", "Cigars", "Walking" ], "children": [ { "name": "Joshua Horuath", "age": 10 } ] }
+{ "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] }
+{ "cid": 89, "name": "Calandra Hedden", "age": 33, "address": { "number": 1231, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Damien Hedden", "age": 19 } ] }
+{ "cid": 90, "name": "Dorethea Korns", "age": null, "address": null, "interests": [ "Cooking", "Computers" ], "children": [ { "name": "Catheryn Korns", "age": 22 } ] }
+{ "cid": 91, "name": "Luna Machen", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Randal Machen", "age": 59 }, { "name": "Emely Machen", "age": null } ] }
+{ "cid": 92, "name": "Kenny Laychock", "age": 15, "address": { "number": 4790, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Basketball" ], "children": [  ] }
+{ "cid": 93, "name": "Garth Raigosa", "age": null, "address": null, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 94, "name": "Edgardo Dunnegan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lyndia Dunnegan", "age": null } ] }
+{ "cid": 95, "name": "Gavin Locey", "age": 86, "address": { "number": 8162, "street": "Lake St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Terrell Locey", "age": null }, { "name": "Kazuko Locey", "age": 36 }, { "name": "Risa Locey", "age": null }, { "name": "Dorethea Locey", "age": 13 } ] }
+{ "cid": 96, "name": "Mara Aument", "age": 72, "address": { "number": 7709, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Cooking", "Movies" ], "children": [ { "name": "Leonardo Aument", "age": 22 } ] }
+{ "cid": 97, "name": "Mui Slosek", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Susanne Slosek", "age": 29 }, { "name": "Colleen Slosek", "age": null } ] }
+{ "cid": 98, "name": "Casimira Hilbrand", "age": 72, "address": { "number": 9693, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Gudrun Hilbrand", "age": 18 }, { "name": "Dacia Hilbrand", "age": 26 }, { "name": "Kortney Hilbrand", "age": null }, { "name": "Luci Hilbrand", "age": null } ] }
+{ "cid": 99, "name": "Bernardina Thacher", "age": 35, "address": { "number": 1582, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Movies", "Fishing", "Fishing" ], "children": [ { "name": "Randee Thacher", "age": null }, { "name": "China Thacher", "age": null } ] }
+{ "cid": 101, "name": "Meaghan Vandel", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Books" ], "children": [ { "name": "Larissa Vandel", "age": null } ] }
+{ "cid": 102, "name": "Melany Rotan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Christiana Rotan", "age": 21 }, { "name": "Lavina Rotan", "age": null }, { "name": "Billy Rotan", "age": null } ] }
+{ "cid": 103, "name": "Rosamond Milera", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 104, "name": "Neda Dilts", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Nona Dilts", "age": 28 }, { "name": "Wm Dilts", "age": null }, { "name": "Svetlana Dilts", "age": 46 }, { "name": "Iva Dilts", "age": 59 } ] }
+{ "cid": 105, "name": "Camilla Lohman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melania Lohman", "age": 50 }, { "name": "Mike Lohman", "age": 53 }, { "name": "Cassaundra Lohman", "age": 32 }, { "name": "Jay Lohman", "age": null } ] }
+{ "cid": 106, "name": "Charles Verna", "age": null, "address": null, "interests": [ "Bass", "Books" ], "children": [ { "name": "Betsy Verna", "age": 37 }, { "name": "Chae Verna", "age": 35 }, { "name": "Naoma Verna", "age": 42 } ] }
+{ "cid": 110, "name": "Karmen Milanesi", "age": 67, "address": { "number": 6223, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash" ], "children": [ { "name": "Emely Milanesi", "age": null }, { "name": "Adam Milanesi", "age": null }, { "name": "Gregg Milanesi", "age": null }, { "name": "Sean Milanesi", "age": 37 } ] }
+{ "cid": 111, "name": "Eddy Ortea", "age": 16, "address": { "number": 6874, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Shera Ortea", "age": null } ] }
+{ "cid": 112, "name": "Dorie Lave", "age": 10, "address": { "number": 2286, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Coffee" ], "children": [ { "name": "Grady Lave", "age": null }, { "name": "Daysi Lave", "age": null } ] }
+{ "cid": 113, "name": "Alayna Daleske", "age": 87, "address": { "number": 4739, "street": "Main St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Hester Daleske", "age": null }, { "name": "Magnolia Daleske", "age": null }, { "name": "Bettye Daleske", "age": 32 } ] }
+{ "cid": 114, "name": "Stephine Capinpin", "age": 78, "address": { "number": 5618, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Puzzles", "Basketball" ], "children": [ { "name": "Krystal Capinpin", "age": 31 }, { "name": "Angelic Capinpin", "age": 45 } ] }
+{ "cid": 115, "name": "Jason Oakden", "age": 89, "address": { "number": 8182, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Music", "Basketball", "Movies" ], "children": [ { "name": "Johnson Oakden", "age": null }, { "name": "Neva Oakden", "age": null }, { "name": "Juliann Oakden", "age": null }, { "name": "Elmer Oakden", "age": null } ] }
+{ "cid": 116, "name": "Conrad Zozaya", "age": 81, "address": { "number": 1667, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Jenette Zozaya", "age": 17 } ] }
+{ "cid": 118, "name": "Ellis Skillom", "age": 78, "address": { "number": 9337, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running", "Cigars" ], "children": [ { "name": "Emory Skillom", "age": null } ] }
+{ "cid": 119, "name": "Chan Morreau", "age": 22, "address": { "number": 1774, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Squash" ], "children": [ { "name": "Arlette Morreau", "age": null } ] }
+{ "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }
+{ "cid": 121, "name": "Shiela Gaustad", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Phebe Gaustad", "age": null }, { "name": "Mavis Gaustad", "age": null }, { "name": "Zula Gaustad", "age": 37 } ] }
+{ "cid": 122, "name": "Wei Perpall", "age": 43, "address": { "number": 916, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ "Bass" ], "children": [ { "name": "Mitchel Perpall", "age": 11 }, { "name": "Aliza Perpall", "age": null }, { "name": "King Perpall", "age": null }, { "name": "Santana Perpall", "age": 22 } ] }
+{ "cid": 123, "name": "Marian Courrege", "age": 30, "address": { "number": 7321, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Coffee" ], "children": [  ] }
+{ "cid": 124, "name": "Kelley Dressman", "age": null, "address": null, "interests": [ "Squash", "Databases", "Fishing" ], "children": [ { "name": "Evie Dressman", "age": null }, { "name": "Fredericka Dressman", "age": null }, { "name": "Leigh Dressman", "age": null }, { "name": "Luna Dressman", "age": 29 } ] }
+{ "cid": 125, "name": "Leigh Pusey", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Elbert Pusey", "age": 44 }, { "name": "Golden Pusey", "age": null }, { "name": "Maria Pusey", "age": null } ] }
+{ "cid": 126, "name": "Grayce Keir", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Antonia Keir", "age": 25 } ] }
+{ "cid": 127, "name": "Christian Anthes", "age": 32, "address": { "number": 6258, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Bass" ], "children": [ { "name": "Sophia Anthes", "age": null } ] }
+{ "cid": 128, "name": "Edwin Harwick", "age": null, "address": null, "interests": [ "Fishing", "Squash", "Basketball" ], "children": [ { "name": "Tomeka Harwick", "age": 34 }, { "name": "Caroline Harwick", "age": 57 }, { "name": "Peter Harwick", "age": null }, { "name": "Adele Harwick", "age": null } ] }
+{ "cid": 129, "name": "Marisha Canzoneri", "age": 84, "address": { "number": 5507, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music", "Databases", "Walking", "Walking" ], "children": [  ] }
+{ "cid": 130, "name": "Kandis Hissem", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Arianna Hissem", "age": null }, { "name": "Necole Hissem", "age": 53 }, { "name": "Manie Hissem", "age": null }, { "name": "Deshawn Hissem", "age": 27 } ] }
+{ "cid": 131, "name": "Kourtney Whitesel", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 134, "name": "Alica Frontiero", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 135, "name": "Josette Dries", "age": null, "address": null, "interests": [ "Base Jumping", "Movies" ], "children": [ { "name": "Ben Dries", "age": 36 }, { "name": "Wm Dries", "age": 29 } ] }
+{ "cid": 136, "name": "Aubrey Kasuboski", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 137, "name": "Camellia Pressman", "age": 81, "address": { "number": 3947, "street": "Park St.", "city": "Seattle" }, "interests": [ "Movies", "Books", "Bass" ], "children": [ { "name": "Dwana Pressman", "age": null }, { "name": "Johnathan Pressman", "age": null }, { "name": "Kasey Pressman", "age": null }, { "name": "Mitch Pressman", "age": null } ] }
+{ "cid": 138, "name": "Ora Villafane", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Deeann Villafane", "age": 22 }, { "name": "Cody Villafane", "age": 47 } ] }
+{ "cid": 139, "name": "Micheline Argenal", "age": null, "address": null, "interests": [ "Bass", "Walking", "Movies" ], "children": [ { "name": "Joye Argenal", "age": 51 }, { "name": "Richard Argenal", "age": 46 }, { "name": "Sarah Argenal", "age": 21 }, { "name": "Jacinda Argenal", "age": 21 } ] }
+{ "cid": 140, "name": "Maryland Neas", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Brunilda Neas", "age": 28 } ] }
+{ "cid": 141, "name": "Adena Klockars", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Bass", "Cigars" ], "children": [  ] }
+{ "cid": 142, "name": "Ervin Softleigh", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Cooking", "Coffee" ], "children": [ { "name": "Russell Softleigh", "age": 50 }, { "name": "Kristy Softleigh", "age": 54 }, { "name": "Refugio Softleigh", "age": null } ] }
+{ "cid": 143, "name": "Katelynn Kanzler", "age": 80, "address": { "number": 9453, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Carl Kanzler", "age": null } ] }
+{ "cid": 144, "name": "Celesta Sosebee", "age": 19, "address": { "number": 2683, "street": "7th St.", "city": "Portland" }, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Jesse Sosebee", "age": null }, { "name": "Oralee Sosebee", "age": null }, { "name": "Sunday Sosebee", "age": null } ] }
+{ "cid": 145, "name": "Carey Bousman", "age": 61, "address": { "number": 16, "street": "Oak St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Lynda Bousman", "age": 32 }, { "name": "Evalyn Bousman", "age": 17 } ] }
+{ "cid": 146, "name": "Glennis Vanruiten", "age": 14, "address": { "number": 8272, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Squash", "Databases" ], "children": [ { "name": "Joanie Vanruiten", "age": null }, { "name": "Long Vanruiten", "age": null }, { "name": "Abdul Vanruiten", "age": null } ] }
+{ "cid": 147, "name": "Marla Pollan", "age": 24, "address": { "number": 9271, "street": "Oak St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Song Pollan", "age": 11 }, { "name": "Lili Pollan", "age": 13 }, { "name": "Shaunte Pollan", "age": 12 }, { "name": "Sandie Pollan", "age": null } ] }
+{ "cid": 148, "name": "Coy Dulay", "age": 66, "address": { "number": 9793, "street": "Hill St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Emile Dulay", "age": null }, { "name": "Letitia Dulay", "age": 38 } ] }
+{ "cid": 149, "name": "Marcella Diamond", "age": 62, "address": { "number": 720, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Ezra Diamond", "age": null } ] }
+{ "cid": 150, "name": "Jesus Vanleeuwen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Sueann Vanleeuwen", "age": 47 }, { "name": "Refugia Vanleeuwen", "age": null }, { "name": "Taisha Vanleeuwen", "age": null }, { "name": "Nathaniel Vanleeuwen", "age": null } ] }
+{ "cid": 151, "name": "Charlyn Soyars", "age": 21, "address": { "number": 2796, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
+{ "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 157, "name": "Mckenzie Tahir", "age": 78, "address": { "number": 6752, "street": "Hill St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Margarita Tahir", "age": 18 }, { "name": "Mia Tahir", "age": 47 }, { "name": "Gaylord Tahir", "age": null } ] }
+{ "cid": 158, "name": "Rosalva Harvath", "age": 84, "address": { "number": 5569, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Wine", "Skiing", "Coffee" ], "children": [ { "name": "Taneka Harvath", "age": null }, { "name": "Ina Harvath", "age": 54 }, { "name": "Joanne Harvath", "age": 51 } ] }
+{ "cid": 159, "name": "Jeanmarie Franchini", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Nikita Franchini", "age": null }, { "name": "Willetta Franchini", "age": null }, { "name": "Ester Franchini", "age": 12 } ] }
+{ "cid": 160, "name": "Yevette Chanez", "age": null, "address": null, "interests": [ "Bass", "Wine", "Coffee" ], "children": [ { "name": "Walter Chanez", "age": 11 }, { "name": "Pa Chanez", "age": 27 } ] }
+{ "cid": 161, "name": "Lucia Tata", "age": 85, "address": { "number": 8058, "street": "Park St.", "city": "Seattle" }, "interests": [ "Basketball", "Bass" ], "children": [ { "name": "Jenifer Tata", "age": 70 }, { "name": "Erna Tata", "age": null } ] }
+{ "cid": 162, "name": "Chang Reek", "age": 85, "address": { "number": 5943, "street": "Washington St.", "city": "Portland" }, "interests": [ "Tennis", "Movies" ], "children": [ { "name": "Camelia Reek", "age": null }, { "name": "Eleonora Reek", "age": 36 }, { "name": "Shalonda Reek", "age": 39 }, { "name": "Stefan Reek", "age": 64 } ] }
+{ "cid": 163, "name": "Marcelene Sparano", "age": 36, "address": { "number": 5722, "street": "View St.", "city": "San Jose" }, "interests": [ "Basketball", "Databases" ], "children": [ { "name": "Luz Sparano", "age": null }, { "name": "Cassandra Sparano", "age": 21 }, { "name": "Martina Sparano", "age": 21 }, { "name": "Elisabeth Sparano", "age": null } ] }
+{ "cid": 164, "name": "Lucrecia Dahlhauser", "age": null, "address": null, "interests": [ "Wine" ], "children": [  ] }
+{ "cid": 165, "name": "Melodie Starrick", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Adria Starrick", "age": null }, { "name": "Tasha Starrick", "age": 25 } ] }
+{ "cid": 166, "name": "Gregorio Plummer", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Santiago Plummer", "age": null }, { "name": "Malisa Plummer", "age": 59 }, { "name": "Tracie Plummer", "age": 40 }, { "name": "Florentina Plummer", "age": 23 } ] }
+{ "cid": 169, "name": "Casandra Fierge", "age": 55, "address": { "number": 175, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 170, "name": "Dana Lese", "age": 38, "address": { "number": 575, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Walking", "Coffee" ], "children": [ { "name": "Yasmine Lese", "age": 24 }, { "name": "Ezekiel Lese", "age": 20 }, { "name": "Ammie Lese", "age": 27 }, { "name": "Robert Lese", "age": 15 } ] }
+{ "cid": 171, "name": "Eddie Shebchuk", "age": 86, "address": { "number": 3304, "street": "Lake St.", "city": "Portland" }, "interests": [ "Books" ], "children": [ { "name": "Harmony Shebchuk", "age": null } ] }
+{ "cid": 172, "name": "Weldon Alquesta", "age": null, "address": null, "interests": [ "Music", "Fishing", "Music" ], "children": [ { "name": "Kip Alquesta", "age": null } ] }
+{ "cid": 173, "name": "Annamae Lucien", "age": 46, "address": { "number": 1253, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Squash" ], "children": [ { "name": "Sanjuana Lucien", "age": 21 }, { "name": "Nathanael Lucien", "age": 27 }, { "name": "Jae Lucien", "age": null }, { "name": "Judith Lucien", "age": null } ] }
+{ "cid": 174, "name": "Taneka Baldassare", "age": 50, "address": { "number": 5787, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Junko Baldassare", "age": null }, { "name": "Denisha Baldassare", "age": null }, { "name": "Hermina Baldassare", "age": 17 }, { "name": "Lexie Baldassare", "age": null } ] }
+{ "cid": 175, "name": "Loise Obhof", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Susann Obhof", "age": null }, { "name": "Signe Obhof", "age": 38 } ] }
+{ "cid": 176, "name": "Kellie Andruszkiewic", "age": null, "address": null, "interests": [ "Fishing", "Puzzles", "Wine", "Skiing" ], "children": [ { "name": "Xiao Andruszkiewic", "age": null }, { "name": "Al Andruszkiewic", "age": 43 } ] }
+{ "cid": 177, "name": "Wilda Hanisch", "age": null, "address": null, "interests": [ "Wine", "Computers" ], "children": [ { "name": "Shannan Hanisch", "age": null }, { "name": "Marissa Hanisch", "age": 30 }, { "name": "Keely Hanisch", "age": 54 }, { "name": "Humberto Hanisch", "age": 17 } ] }
+{ "cid": 178, "name": "Athena Kaluna", "age": null, "address": null, "interests": [ "Running", "Computers", "Basketball" ], "children": [ { "name": "Rosalba Kaluna", "age": 48 }, { "name": "Max Kaluna", "age": 10 } ] }
+{ "cid": 179, "name": "Antonette Bernice", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Solange Bernice", "age": null } ] }
+{ "cid": 180, "name": "Theda Hilz", "age": 35, "address": { "number": 9918, "street": "Oak St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Ethan Hilz", "age": null }, { "name": "Bill Hilz", "age": 12 } ] }
+{ "cid": 181, "name": "Toni Sanghani", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Hollie Sanghani", "age": 29 } ] }
+{ "cid": 182, "name": "Christiana Westlie", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Ilda Westlie", "age": 18 } ] }
+{ "cid": 183, "name": "Ladawn Vyas", "age": 64, "address": { "number": 2663, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 184, "name": "Mirtha Ricciardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Elsa Ricciardi", "age": 30 }, { "name": "Vicente Ricciardi", "age": null }, { "name": "Sau Ricciardi", "age": 28 } ] }
+{ "cid": 185, "name": "Abigail Zugg", "age": 22, "address": { "number": 6676, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Video Games", "Basketball" ], "children": [ { "name": "Peter Zugg", "age": 10 }, { "name": "Ariane Zugg", "age": null } ] }
+{ "cid": 187, "name": "Seema Hartsch", "age": 80, "address": { "number": 6629, "street": "Lake St.", "city": "Portland" }, "interests": [ "Coffee", "Coffee", "Cigars" ], "children": [ { "name": "Suellen Hartsch", "age": null }, { "name": "Pennie Hartsch", "age": 20 }, { "name": "Aubrey Hartsch", "age": null }, { "name": "Randy Hartsch", "age": 32 } ] }
+{ "cid": 188, "name": "Brynn Bendorf", "age": 23, "address": { "number": 1168, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Leesa Bendorf", "age": 11 }, { "name": "Daine Bendorf", "age": null } ] }
+{ "cid": 189, "name": "Shyla Saathoff", "age": 85, "address": { "number": 9679, "street": "Main St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Johanne Saathoff", "age": 61 }, { "name": "Janett Saathoff", "age": null } ] }
+{ "cid": 190, "name": "Kristel Axelson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Deja Axelson", "age": null } ] }
+{ "cid": 191, "name": "Lula Pangburn", "age": 42, "address": { "number": 1309, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Skiing", "Cooking", "Walking", "Video Games" ], "children": [ { "name": "Love Pangburn", "age": 11 }, { "name": "Bryant Pangburn", "age": 13 }, { "name": "Kenda Pangburn", "age": 14 } ] }
+{ "cid": 193, "name": "Melisa Maccarter", "age": 50, "address": { "number": 1494, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [ { "name": "Yetta Maccarter", "age": null }, { "name": "Geralyn Maccarter", "age": null } ] }
+{ "cid": 194, "name": "Leslee Apking", "age": 41, "address": { "number": 8107, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Irena Apking", "age": null }, { "name": "Arla Apking", "age": null } ] }
+{ "cid": 195, "name": "Annetta Demille", "age": 17, "address": { "number": 5722, "street": "Park St.", "city": "Portland" }, "interests": [ "Bass" ], "children": [ { "name": "Natacha Demille", "age": null }, { "name": "Giuseppe Demille", "age": null }, { "name": "Kami Demille", "age": null }, { "name": "Jewell Demille", "age": null } ] }
+{ "cid": 196, "name": "Darwin Seekell", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Kathryne Seekell", "age": null }, { "name": "Marlon Seekell", "age": null }, { "name": "Shiloh Seekell", "age": 51 } ] }
+{ "cid": 197, "name": "Garth Giannitti", "age": null, "address": null, "interests": [ "Coffee", "Cigars" ], "children": [ { "name": "Patsy Giannitti", "age": null }, { "name": "Ray Giannitti", "age": 35 }, { "name": "Kamala Giannitti", "age": 35 }, { "name": "Lauran Giannitti", "age": 25 } ] }
+{ "cid": 198, "name": "Thelma Youkers", "age": null, "address": null, "interests": [ "Basketball", "Movies", "Cooking" ], "children": [ { "name": "Shamika Youkers", "age": 28 } ] }
+{ "cid": 199, "name": "Rogelio Hannan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Blanche Hannan", "age": null }, { "name": "Elvira Hannan", "age": null }, { "name": "Cinderella Hannan", "age": null } ] }
+{ "cid": 200, "name": "Stacey Bertran", "age": 78, "address": { "number": 9050, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Eugenia Bertran", "age": 59 }, { "name": "Lorri Bertran", "age": 29 }, { "name": "Corrie Bertran", "age": 52 } ] }
+{ "cid": 201, "name": "Tiny Hoysradt", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Simon Hoysradt", "age": 24 } ] }
+{ "cid": 202, "name": "Evangelina Poloskey", "age": 46, "address": { "number": 8285, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Anthony Poloskey", "age": 27 }, { "name": "Olga Poloskey", "age": 10 }, { "name": "Carmon Poloskey", "age": 13 }, { "name": "Tanja Poloskey", "age": 20 } ] }
+{ "cid": 203, "name": "Elke Mazurowski", "age": 52, "address": { "number": 9276, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Esta Mazurowski", "age": null }, { "name": "Clarence Mazurowski", "age": 14 } ] }
+{ "cid": 204, "name": "Londa Herdt", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marnie Herdt", "age": 47 } ] }
+{ "cid": 205, "name": "Moises Plake", "age": null, "address": null, "interests": [ "Puzzles", "Computers" ], "children": [  ] }
+{ "cid": 206, "name": "Armand Hauersperger", "age": 67, "address": { "number": 7266, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Charlott Hauersperger", "age": 47 }, { "name": "Kayla Hauersperger", "age": null }, { "name": "Maris Hauersperger", "age": 52 } ] }
+{ "cid": 207, "name": "Phyliss Honda", "age": 22, "address": { "number": 8387, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Cooking", "Music", "Books" ], "children": [ { "name": "Bee Honda", "age": null }, { "name": "Cyril Honda", "age": null }, { "name": "Vertie Honda", "age": null } ] }
+{ "cid": 210, "name": "Jillian Roadruck", "age": null, "address": null, "interests": [ "Coffee", "Tennis" ], "children": [ { "name": "Marguerite Roadruck", "age": null }, { "name": "Ilana Roadruck", "age": null }, { "name": "Chantelle Roadruck", "age": 19 }, { "name": "Nikia Roadruck", "age": 43 } ] }
+{ "cid": 211, "name": "Kristian Knepshield", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 212, "name": "Christi Vichi", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 213, "name": "Micheal Evoy", "age": 68, "address": { "number": 1219, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Skiing", "Computers", "Books", "Puzzles" ], "children": [ { "name": "Socorro Evoy", "age": null }, { "name": "Gertude Evoy", "age": 36 }, { "name": "Araceli Evoy", "age": null }, { "name": "Yasmin Evoy", "age": null } ] }
+{ "cid": 214, "name": "Louvenia Zaffalon", "age": null, "address": null, "interests": [ "Skiing", "Books" ], "children": [  ] }
+{ "cid": 215, "name": "Ashton Schadegg", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ciara Schadegg", "age": null }, { "name": "Karisa Schadegg", "age": 11 }, { "name": "Hayden Schadegg", "age": 44 } ] }
+{ "cid": 216, "name": "Odilia Lampson", "age": null, "address": null, "interests": [ "Wine", "Databases", "Basketball" ], "children": [ { "name": "Callie Lampson", "age": null } ] }
+{ "cid": 217, "name": "Scott Fulks", "age": null, "address": null, "interests": [ "Computers" ], "children": [  ] }
+{ "cid": 218, "name": "Clarinda Stagliano", "age": 76, "address": { "number": 3258, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Cigars" ], "children": [  ] }
+{ "cid": 219, "name": "Joelle Valazquez", "age": 73, "address": { "number": 9775, "street": "Park St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Gene Valazquez", "age": null }, { "name": "Ilona Valazquez", "age": null } ] }
+{ "cid": 220, "name": "Soila Hannemann", "age": null, "address": null, "interests": [ "Wine", "Puzzles", "Basketball" ], "children": [ { "name": "Piper Hannemann", "age": 44 } ] }
+{ "cid": 221, "name": "Delois Fiqueroa", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Cherri Fiqueroa", "age": null } ] }
+{ "cid": 222, "name": "Malcom Bloomgren", "age": 39, "address": { "number": 4674, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Rosia Bloomgren", "age": null }, { "name": "Bryant Bloomgren", "age": 15 }, { "name": "Donnie Bloomgren", "age": null } ] }
+{ "cid": 223, "name": "Margurite Embelton", "age": 19, "address": { "number": 554, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Sherie Embelton", "age": null }, { "name": "Monica Embelton", "age": null }, { "name": "Jeanne Embelton", "age": null }, { "name": "Santiago Embelton", "age": null } ] }
+{ "cid": 224, "name": "Rene Rowey", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "children": [ { "name": "Necole Rowey", "age": 26 }, { "name": "Sharyl Rowey", "age": 20 }, { "name": "Yvone Rowey", "age": 36 } ] }
+{ "cid": 225, "name": "Shantel Drapeaux", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Felicidad Drapeaux", "age": null }, { "name": "Wanetta Drapeaux", "age": 52 }, { "name": "Louise Drapeaux", "age": 28 }, { "name": "Pat Drapeaux", "age": null } ] }
+{ "cid": 226, "name": "Debrah Deppert", "age": 62, "address": { "number": 7699, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Tonie Deppert", "age": 25 }, { "name": "Neil Deppert", "age": null } ] }
+{ "cid": 227, "name": "Carlos Skyes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Cortney Skyes", "age": 32 } ] }
+{ "cid": 228, "name": "Donnette Brumbley", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Madlyn Brumbley", "age": null }, { "name": "Apolonia Brumbley", "age": 13 }, { "name": "Stephine Brumbley", "age": null }, { "name": "Zelma Brumbley", "age": 51 } ] }
+{ "cid": 229, "name": "Raymundo Meurin", "age": null, "address": null, "interests": [ "Bass", "Basketball", "Databases" ], "children": [ { "name": "Mariela Meurin", "age": null } ] }
+{ "cid": 230, "name": "Tobias Vicars", "age": 66, "address": { "number": 638, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Books", "Walking" ], "children": [  ] }
+{ "cid": 231, "name": "Arianne Wedlow", "age": 68, "address": { "number": 9663, "street": "7th St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Birdie Wedlow", "age": 32 }, { "name": "Pearle Wedlow", "age": 13 }, { "name": "Jordon Wedlow", "age": 43 }, { "name": "Katherin Wedlow", "age": 18 } ] }
+{ "cid": 232, "name": "Joey Potes", "age": null, "address": null, "interests": [ "Bass", "Bass", "Base Jumping" ], "children": [ { "name": "Bobby Potes", "age": null } ] }
+{ "cid": 233, "name": "Sammy Coalter", "age": null, "address": null, "interests": [ "Fishing", "Base Jumping" ], "children": [ { "name": "Twana Coalter", "age": null }, { "name": "Nenita Coalter", "age": 30 } ] }
+{ "cid": 234, "name": "Ilana Brothern", "age": 36, "address": { "number": 4850, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles", "Walking", "Fishing" ], "children": [ { "name": "Shayne Brothern", "age": null }, { "name": "Phillis Brothern", "age": null } ] }
+{ "cid": 235, "name": "Orpha Craycraft", "age": null, "address": null, "interests": [ "Skiing", "Squash" ], "children": [  ] }
+{ "cid": 236, "name": "Muriel Laib", "age": 25, "address": { "number": 4481, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Tennis" ], "children": [ { "name": "Jann Laib", "age": null }, { "name": "Lila Laib", "age": 10 }, { "name": "Elyse Laib", "age": 11 } ] }
+{ "cid": 237, "name": "Sona Hehn", "age": 47, "address": { "number": 3720, "street": "Oak St.", "city": "Portland" }, "interests": [ "Computers", "Squash", "Coffee" ], "children": [ { "name": "Marquerite Hehn", "age": null }, { "name": "Suellen Hehn", "age": 29 }, { "name": "Herb Hehn", "age": 29 } ] }
+{ "cid": 238, "name": "Marcelina Redic", "age": null, "address": null, "interests": [ "Cigars", "Cigars", "Coffee" ], "children": [ { "name": "Renate Redic", "age": null }, { "name": "Kyoko Redic", "age": null }, { "name": "Dorthey Redic", "age": null } ] }
+{ "cid": 239, "name": "Celsa Fondow", "age": null, "address": null, "interests": [ "Base Jumping", "Computers", "Cooking", "Wine" ], "children": [  ] }
+{ "cid": 241, "name": "Lesha Ambrosia", "age": 49, "address": { "number": 6133, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running" ], "children": [ { "name": "Venice Ambrosia", "age": null } ] }
+{ "cid": 242, "name": "Jerold Shabot", "age": null, "address": null, "interests": [ "Fishing", "Walking", "Walking", "Puzzles" ], "children": [ { "name": "Marie Shabot", "age": 26 } ] }
+{ "cid": 243, "name": "Love Hoftiezer", "age": 88, "address": { "number": 2491, "street": "Main St.", "city": "Portland" }, "interests": [ "Cigars", "Coffee", "Books" ], "children": [ { "name": "Kellee Hoftiezer", "age": 77 } ] }
+{ "cid": 244, "name": "Rene Shenk", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Skiing" ], "children": [ { "name": "Victor Shenk", "age": 28 }, { "name": "Doris Shenk", "age": null }, { "name": "Max Shenk", "age": 51 } ] }
+{ "cid": 245, "name": "Lupe Abshear", "age": 55, "address": { "number": 7269, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Song Abshear", "age": null }, { "name": "Honey Abshear", "age": 31 } ] }
+{ "cid": 246, "name": "Kenda Heikkinen", "age": 63, "address": { "number": 8924, "street": "View St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 247, "name": "Minda Heron", "age": 25, "address": { "number": 1629, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 249, "name": "Kiana Satiago", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Stacy Satiago", "age": null } ] }
+{ "cid": 250, "name": "Angeles Saltonstall", "age": null, "address": null, "interests": [ "Tennis", "Fishing", "Movies" ], "children": [ { "name": "Suzanna Saltonstall", "age": null } ] }
+{ "cid": 251, "name": "Janeen Galston", "age": null, "address": null, "interests": [ "Basketball", "Base Jumping" ], "children": [  ] }
+{ "cid": 252, "name": "Almeda Charity", "age": 19, "address": { "number": 5553, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Rosia Charity", "age": null } ] }
+{ "cid": 254, "name": "Jeanice Longanecker", "age": 74, "address": { "number": 2613, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Books", "Base Jumping" ], "children": [  ] }
+{ "cid": 255, "name": "Cherri Piegaro", "age": 64, "address": { "number": 3802, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Elwood Piegaro", "age": null } ] }
+{ "cid": 256, "name": "Chester Rosenberg", "age": 46, "address": { "number": 8673, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Basketball" ], "children": [ { "name": "Gemma Rosenberg", "age": null }, { "name": "Marty Rosenberg", "age": null } ] }
+{ "cid": 257, "name": "Altha Jastrzebski", "age": 21, "address": { "number": 4405, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 258, "name": "Florentina Hense", "age": 20, "address": { "number": 8495, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Noelle Hense", "age": null }, { "name": "Roxann Hense", "age": null } ] }
+{ "cid": 259, "name": "Aurelio Darrigo", "age": 45, "address": { "number": 1114, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Leonard Darrigo", "age": 22 }, { "name": "Aron Darrigo", "age": null }, { "name": "Pamelia Darrigo", "age": 14 } ] }
+{ "cid": 260, "name": "Hedwig Caminero", "age": 81, "address": { "number": 4305, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Databases" ], "children": [ { "name": "Hal Caminero", "age": null }, { "name": "Cierra Caminero", "age": 32 } ] }
+{ "cid": 263, "name": "Mellisa Machalek", "age": null, "address": null, "interests": [ "Bass", "Coffee", "Skiing" ], "children": [  ] }
+{ "cid": 264, "name": "Leon Yoshizawa", "age": 81, "address": { "number": 608, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Running", "Books", "Running" ], "children": [ { "name": "Carmela Yoshizawa", "age": 34 } ] }
+{ "cid": 265, "name": "Donte Stempien", "age": 25, "address": { "number": 3882, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine", "Books" ], "children": [  ] }
+{ "cid": 266, "name": "Carlee Friddle", "age": 74, "address": { "number": 6538, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Candie Friddle", "age": null }, { "name": "Zoila Friddle", "age": 59 } ] }
+{ "cid": 267, "name": "Renay Huddelston", "age": 68, "address": { "number": 1939, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Wine", "Base Jumping" ], "children": [ { "name": "Colene Huddelston", "age": null } ] }
+{ "cid": 268, "name": "Fernando Pingel", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Books" ], "children": [ { "name": "Latrice Pingel", "age": null }, { "name": "Wade Pingel", "age": 13 }, { "name": "Christal Pingel", "age": null }, { "name": "Melania Pingel", "age": null } ] }
+{ "cid": 269, "name": "Dante Sharko", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Ahmad Sharko", "age": 34 }, { "name": "Mona Sharko", "age": null }, { "name": "Stephaine Sharko", "age": 42 }, { "name": "Adrianna Sharko", "age": null } ] }
+{ "cid": 270, "name": "Lavon Ascenzo", "age": null, "address": null, "interests": [ "Books", "Skiing" ], "children": [  ] }
+{ "cid": 271, "name": "Carey Ronin", "age": 44, "address": { "number": 8141, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Cigars", "Video Games" ], "children": [ { "name": "Lonny Ronin", "age": null }, { "name": "Armanda Ronin", "age": null } ] }
+{ "cid": 272, "name": "Frederick Valla", "age": 15, "address": { "number": 6805, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Carroll Valla", "age": null } ] }
+{ "cid": 273, "name": "Corrinne Seaquist", "age": 24, "address": { "number": 6712, "street": "7th St.", "city": "Portland" }, "interests": [ "Puzzles", "Coffee", "Wine" ], "children": [ { "name": "Mignon Seaquist", "age": null }, { "name": "Leo Seaquist", "age": null } ] }
+{ "cid": 274, "name": "Claude Harral", "age": null, "address": null, "interests": [ "Squash", "Bass", "Cooking" ], "children": [ { "name": "Archie Harral", "age": null }, { "name": "Royal Harral", "age": null } ] }
+{ "cid": 275, "name": "Natalie Ifeanyi", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] }
+{ "cid": 277, "name": "Malena Smock", "age": null, "address": null, "interests": [ "Running", "Base Jumping" ], "children": [ { "name": "Inocencia Smock", "age": 50 }, { "name": "Cleveland Smock", "age": null } ] }
+{ "cid": 278, "name": "Deb Nicole", "age": 59, "address": { "number": 9003, "street": "Park St.", "city": "Seattle" }, "interests": [ "Books", "Computers", "Walking", "Cooking" ], "children": [ { "name": "Len Nicole", "age": null } ] }
+{ "cid": 279, "name": "Saundra Croan", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Jena Croan", "age": 37 }, { "name": "Sarai Croan", "age": null }, { "name": "Junita Croan", "age": null }, { "name": "Ferdinand Croan", "age": 43 } ] }
+{ "cid": 280, "name": "Marlo Maung", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Harold Maung", "age": null } ] }
+{ "cid": 282, "name": "Emelda Dawood", "age": 32, "address": { "number": 5261, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Venus Dawood", "age": 12 }, { "name": "Gertrude Dawood", "age": null }, { "name": "Yen Dawood", "age": null }, { "name": "Theresa Dawood", "age": 16 } ] }
+{ "cid": 283, "name": "Pilar Fritts", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Jeneva Fritts", "age": null }, { "name": "Gail Fritts", "age": 25 } ] }
+{ "cid": 285, "name": "Edgar Farlin", "age": 75, "address": { "number": 3833, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Databases" ], "children": [ { "name": "Stefanie Farlin", "age": 60 }, { "name": "Catina Farlin", "age": null }, { "name": "Lizzie Farlin", "age": null }, { "name": "Beau Farlin", "age": null } ] }
+{ "cid": 286, "name": "Tara Sioma", "age": 18, "address": { "number": 9425, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Dawna Sioma", "age": null }, { "name": "Jeanne Sioma", "age": null } ] }
+{ "cid": 288, "name": "Sharice Bachicha", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 289, "name": "Clarence Milette", "age": 16, "address": { "number": 3778, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Books", "Base Jumping", "Music" ], "children": [  ] }
+{ "cid": 290, "name": "Kimberly Gullatte", "age": 51, "address": { "number": 4130, "street": "Park St.", "city": "San Jose" }, "interests": [ "Running", "Squash", "Databases" ], "children": [ { "name": "Micheal Gullatte", "age": null }, { "name": "Estrella Gullatte", "age": 40 }, { "name": "Corrine Gullatte", "age": null }, { "name": "Ward Gullatte", "age": null } ] }
+{ "cid": 291, "name": "Svetlana Moone", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Running", "Walking" ], "children": [ { "name": "Emelina Moone", "age": null }, { "name": "Candi Moone", "age": null } ] }
+{ "cid": 292, "name": "Mariana Cosselman", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Madge Cosselman", "age": 43 } ] }
+{ "cid": 293, "name": "Terresa Hofstetter", "age": 15, "address": { "number": 3338, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Computers", "Running", "Cigars", "Fishing" ], "children": [ { "name": "Hubert Hofstetter", "age": null }, { "name": "Jolie Hofstetter", "age": null } ] }
+{ "cid": 294, "name": "Foster Salimi", "age": 79, "address": { "number": 8439, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Pei Salimi", "age": null } ] }
+{ "cid": 295, "name": "Guillermina Florek", "age": 61, "address": { "number": 3704, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies", "Books" ], "children": [ { "name": "Donnie Florek", "age": null }, { "name": "Jeannetta Florek", "age": 38 }, { "name": "Leigha Florek", "age": null }, { "name": "Zenobia Florek", "age": 10 } ] }
+{ "cid": 296, "name": "Doreen Kea", "age": 89, "address": { "number": 7034, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Lyndsay Kea", "age": 68 }, { "name": "Trena Kea", "age": 18 } ] }
+{ "cid": 297, "name": "Adeline Frierson", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Fishing" ], "children": [ { "name": "Marci Frierson", "age": null }, { "name": "Rolanda Frierson", "age": null }, { "name": "Del Frierson", "age": null } ] }
+{ "cid": 298, "name": "Brittny Christin", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Hilario Christin", "age": null }, { "name": "Clarine Christin", "age": null } ] }
+{ "cid": 299, "name": "Jacob Wainman", "age": 76, "address": { "number": 4551, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Wine", "Coffee" ], "children": [ { "name": "Abram Wainman", "age": 28 }, { "name": "Ramonita Wainman", "age": 18 }, { "name": "Sheryll Wainman", "age": null } ] }
+{ "cid": 300, "name": "Garret Colgrove", "age": 85, "address": { "number": 9937, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Puzzles", "Fishing" ], "children": [ { "name": "Janna Colgrove", "age": null }, { "name": "Jerilyn Colgrove", "age": 35 } ] }
+{ "cid": 301, "name": "Cherry Steenwyk", "age": 88, "address": { "number": 4138, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Movies" ], "children": [ { "name": "Toccara Steenwyk", "age": 66 }, { "name": "Tari Steenwyk", "age": null }, { "name": "Lawanna Steenwyk", "age": null }, { "name": "Ossie Steenwyk", "age": 26 } ] }
+{ "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] }
+{ "cid": 303, "name": "Michel Bayird", "age": 37, "address": { "number": 7939, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Shan Bayird", "age": 12 } ] }
+{ "cid": 304, "name": "Francine Reddin", "age": 39, "address": { "number": 9392, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Millicent Reddin", "age": null } ] }
+{ "cid": 305, "name": "Tuyet Leinbach", "age": null, "address": null, "interests": [ "Puzzles", "Walking" ], "children": [  ] }
+{ "cid": 306, "name": "Laurie Tuff", "age": null, "address": null, "interests": [ "Computers", "Base Jumping", "Bass", "Basketball" ], "children": [ { "name": "Sharie Tuff", "age": null }, { "name": "Ollie Tuff", "age": 53 }, { "name": "Gonzalo Tuff", "age": null }, { "name": "Thomas Tuff", "age": null } ] }
+{ "cid": 307, "name": "Abraham Lanphear", "age": 20, "address": { "number": 7552, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Toccara Lanphear", "age": null }, { "name": "Milly Lanphear", "age": null } ] }
+{ "cid": 308, "name": "Solomon Schwenke", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Gertrude Schwenke", "age": null }, { "name": "Marcell Schwenke", "age": 41 }, { "name": "Shalon Schwenke", "age": null } ] }
+{ "cid": 309, "name": "Lise Baiz", "age": 46, "address": { "number": 352, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Alisa Baiz", "age": 18 }, { "name": "Elidia Baiz", "age": 28 }, { "name": "Ray Baiz", "age": 19 } ] }
+{ "cid": 311, "name": "Ria Haflett", "age": 14, "address": { "number": 9513, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Jimmie Haflett", "age": null }, { "name": "Dario Haflett", "age": null }, { "name": "Robbyn Haflett", "age": null } ] }
+{ "cid": 312, "name": "Epifania Chorney", "age": 62, "address": { "number": 9749, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Puzzles", "Tennis" ], "children": [ { "name": "Lizeth Chorney", "age": 22 } ] }
+{ "cid": 313, "name": "Lasandra Raigosa", "age": null, "address": null, "interests": [ "Walking", "Walking" ], "children": [ { "name": "Lanelle Raigosa", "age": null } ] }
+{ "cid": 314, "name": "Gwendolyn Abeb", "age": 85, "address": { "number": 3977, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Basketball", "Music", "Squash", "Walking" ], "children": [ { "name": "Aurelia Abeb", "age": 14 }, { "name": "Young Abeb", "age": null }, { "name": "Shay Abeb", "age": null }, { "name": "Lavina Abeb", "age": 15 } ] }
+{ "cid": 315, "name": "Kallie Eiselein", "age": null, "address": null, "interests": [ "Computers", "Tennis" ], "children": [  ] }
+{ "cid": 316, "name": "Patrina Whitting", "age": 74, "address": { "number": 4772, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Video Games", "Bass" ], "children": [ { "name": "Rubye Whitting", "age": null } ] }
+{ "cid": 317, "name": "Zona Caffarel", "age": 52, "address": { "number": 9419, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Cortez Caffarel", "age": null } ] }
+{ "cid": 318, "name": "Shaunna Royal", "age": 86, "address": { "number": 8681, "street": "7th St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Shantell Royal", "age": 37 }, { "name": "Shalon Royal", "age": 50 }, { "name": "Chung Royal", "age": 26 } ] }
+{ "cid": 319, "name": "Ashlie Rott", "age": 42, "address": { "number": 366, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Computers", "Cooking", "Databases" ], "children": [  ] }
+{ "cid": 320, "name": "Charley Hermenegildo", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melda Hermenegildo", "age": 51 }, { "name": "Lashon Hermenegildo", "age": null } ] }
+{ "cid": 322, "name": "Jaclyn Ettl", "age": 83, "address": { "number": 4500, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Noah Ettl", "age": 30 }, { "name": "Kesha Ettl", "age": null } ] }
+{ "cid": 323, "name": "Rebeca Grisostomo", "age": 26, "address": { "number": 399, "street": "View St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Iva Grisostomo", "age": 12 }, { "name": "Ha Grisostomo", "age": null }, { "name": "Lorna Grisostomo", "age": null } ] }
+{ "cid": 324, "name": "Wendolyn Centorino", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 325, "name": "Ai Tarleton", "age": null, "address": null, "interests": [ "Coffee", "Music" ], "children": [ { "name": "Risa Tarleton", "age": 24 }, { "name": "Leonila Tarleton", "age": null }, { "name": "Thomasina Tarleton", "age": null } ] }
+{ "cid": 326, "name": "Tad Tellers", "age": null, "address": null, "interests": [ "Books", "Tennis", "Base Jumping" ], "children": [ { "name": "Fannie Tellers", "age": null } ] }
+{ "cid": 327, "name": "Minnie Scali", "age": null, "address": null, "interests": [ "Cooking", "Squash", "Skiing" ], "children": [ { "name": "Jalisa Scali", "age": null }, { "name": "Preston Scali", "age": null }, { "name": "Stephani Scali", "age": 47 }, { "name": "Candra Scali", "age": null } ] }
+{ "cid": 328, "name": "Mallory Sheffey", "age": 27, "address": { "number": 8532, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Cooking" ], "children": [ { "name": "Regan Sheffey", "age": 14 } ] }
+{ "cid": 330, "name": "Noma Tollefsen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melody Tollefsen", "age": 45 }, { "name": "Caridad Tollefsen", "age": 15 } ] }
+{ "cid": 331, "name": "Willena Provenza", "age": 43, "address": { "number": 6742, "street": "Main St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Alesha Provenza", "age": 32 }, { "name": "Marty Provenza", "age": null }, { "name": "Lindy Provenza", "age": 21 }, { "name": "Junita Provenza", "age": null } ] }
+{ "cid": 332, "name": "Malcom Cafasso", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marie Cafasso", "age": null }, { "name": "Asley Cafasso", "age": 38 } ] }
+{ "cid": 333, "name": "Conchita Olivera", "age": 37, "address": { "number": 8519, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Trenton Olivera", "age": null }, { "name": "Shin Olivera", "age": 26 }, { "name": "Everett Olivera", "age": 15 }, { "name": "Shera Olivera", "age": 20 } ] }
+{ "cid": 335, "name": "Odessa Dammeyer", "age": 18, "address": { "number": 6828, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Basketball", "Bass", "Cigars" ], "children": [ { "name": "Lindsey Dammeyer", "age": null } ] }
+{ "cid": 336, "name": "Jalisa Talamantez", "age": 78, "address": { "number": 9902, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Video Games", "Squash" ], "children": [  ] }
+{ "cid": 337, "name": "Kay Durney", "age": 52, "address": { "number": 4203, "street": "View St.", "city": "Seattle" }, "interests": [ "Walking" ], "children": [ { "name": "Velia Durney", "age": 38 }, { "name": "Erin Durney", "age": null } ] }
+{ "cid": 338, "name": "Dorthey Roncskevitz", "age": 38, "address": { "number": 4366, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Mindy Roncskevitz", "age": null } ] }
+{ "cid": 339, "name": "Sharonda Catalino", "age": 15, "address": { "number": 7616, "street": "Washington St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Lorine Catalino", "age": null } ] }
+{ "cid": 340, "name": "Erick Faiola", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Marquita Faiola", "age": null }, { "name": "Tasia Faiola", "age": null }, { "name": "Micheal Faiola", "age": 24 }, { "name": "Salvatore Faiola", "age": null } ] }
+{ "cid": 343, "name": "Kaylee Ozaine", "age": 78, "address": { "number": 3367, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Darwin Ozaine", "age": 35 }, { "name": "Anne Ozaine", "age": 13 }, { "name": "Kenneth Ozaine", "age": null }, { "name": "Pat Ozaine", "age": 53 } ] }
+{ "cid": 346, "name": "Elden Choma", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Valorie Choma", "age": null }, { "name": "Leslee Choma", "age": null } ] }
+{ "cid": 347, "name": "Patrick Feighan", "age": 34, "address": { "number": 7613, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Madaline Feighan", "age": null } ] }
+{ "cid": 348, "name": "Matthew Pantaleo", "age": 80, "address": { "number": 9782, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Faviola Pantaleo", "age": null }, { "name": "Yang Pantaleo", "age": null }, { "name": "Christopher Pantaleo", "age": null }, { "name": "Jacqui Pantaleo", "age": 58 } ] }
+{ "cid": 349, "name": "Cristine Hila", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Nyla Hila", "age": 51 } ] }
+{ "cid": 352, "name": "Bonny Sischo", "age": null, "address": null, "interests": [ "Bass", "Movies", "Computers" ], "children": [ { "name": "Judith Sischo", "age": 43 }, { "name": "Adeline Sischo", "age": null }, { "name": "Dayna Sischo", "age": null } ] }
+{ "cid": 353, "name": "Melody Bernas", "age": 76, "address": { "number": 6783, "street": "Main St.", "city": "San Jose" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Kristel Bernas", "age": 45 }, { "name": "Clorinda Bernas", "age": 10 }, { "name": "Natosha Bernas", "age": null } ] }
+{ "cid": 354, "name": "Marian Munzell", "age": 73, "address": { "number": 4504, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Puzzles" ], "children": [  ] }
+{ "cid": 355, "name": "Elois Leckband", "age": null, "address": null, "interests": [ "Skiing", "Wine" ], "children": [  ] }
+{ "cid": 356, "name": "Pearlene Sakumoto", "age": 22, "address": { "number": 5895, "street": "7th St.", "city": "San Jose" }, "interests": [ "Computers", "Bass", "Base Jumping", "Coffee" ], "children": [  ] }
+{ "cid": 357, "name": "Dario Lobach", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Kendall Lobach", "age": 37 } ] }
+{ "cid": 358, "name": "Fredricka Krum", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Darrick Krum", "age": null }, { "name": "Julieann Krum", "age": null }, { "name": "Sun Krum", "age": null }, { "name": "Rosamaria Krum", "age": 16 } ] }
+{ "cid": 360, "name": "Billye Grumet", "age": 82, "address": { "number": 7052, "street": "Main St.", "city": "Portland" }, "interests": [ "Coffee" ], "children": [ { "name": "Linnea Grumet", "age": null }, { "name": "Charline Grumet", "age": 67 } ] }
+{ "cid": 361, "name": "Angela Lacki", "age": 35, "address": { "number": 9710, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Skiing" ], "children": [  ] }
+{ "cid": 362, "name": "Alta Bantug", "age": null, "address": null, "interests": [ "Computers" ], "children": [  ] }
+{ "cid": 363, "name": "Merlene Hoying", "age": 25, "address": { "number": 2105, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash", "Music" ], "children": [ { "name": "Andrew Hoying", "age": 10 } ] }
+{ "cid": 364, "name": "Joni Dazey", "age": 14, "address": { "number": 1237, "street": "Oak St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Kraig Dazey", "age": null } ] }
+{ "cid": 366, "name": "Rosia Wenzinger", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 367, "name": "Cassondra Fabiani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Evia Fabiani", "age": null }, { "name": "Chaya Fabiani", "age": null }, { "name": "Sherman Fabiani", "age": null }, { "name": "Kathi Fabiani", "age": 54 } ] }
+{ "cid": 368, "name": "Tequila Scandalios", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nilsa Scandalios", "age": null }, { "name": "Kaye Scandalios", "age": 23 }, { "name": "Angelo Scandalios", "age": 24 } ] }
+{ "cid": 369, "name": "Nickole Dory", "age": 10, "address": { "number": 4761, "street": "View St.", "city": "Portland" }, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Annmarie Dory", "age": null }, { "name": "Michele Dory", "age": null }, { "name": "Annamae Dory", "age": null }, { "name": "Flora Dory", "age": null } ] }
+{ "cid": 370, "name": "Shonta Furby", "age": 18, "address": { "number": 5792, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Raleigh Furby", "age": null }, { "name": "Britta Furby", "age": null }, { "name": "Gay Furby", "age": null }, { "name": "Elenor Furby", "age": null } ] }
+{ "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] }
+{ "cid": 372, "name": "Zena Keglovic", "age": 22, "address": { "number": 7675, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Wine" ], "children": [  ] }
+{ "cid": 373, "name": "Heather Seward", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Glinda Seward", "age": 59 }, { "name": "Maribeth Seward", "age": null }, { "name": "Teofila Seward", "age": null }, { "name": "Clemencia Seward", "age": 38 } ] }
+{ "cid": 374, "name": "Clair Quinn", "age": null, "address": null, "interests": [ "Walking", "Books" ], "children": [ { "name": "Wesley Quinn", "age": 17 }, { "name": "Maren Quinn", "age": 50 }, { "name": "Ila Quinn", "age": 43 }, { "name": "Casie Quinn", "age": null } ] }
+{ "cid": 375, "name": "Chia Sagaser", "age": 15, "address": { "number": 6025, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing" ], "children": [ { "name": "Garnet Sagaser", "age": null }, { "name": "Mario Sagaser", "age": null }, { "name": "Sun Sagaser", "age": null } ] }
+{ "cid": 376, "name": "Jeffrey Hegarty", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "April Hegarty", "age": null }, { "name": "Wilbur Hegarty", "age": null }, { "name": "Hanh Hegarty", "age": null } ] }
+{ "cid": 377, "name": "Zona Klint", "age": 22, "address": { "number": 6320, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Evie Klint", "age": null }, { "name": "Sharyl Klint", "age": 11 }, { "name": "Joaquina Klint", "age": 11 }, { "name": "Doloris Klint", "age": 11 } ] }
+{ "cid": 378, "name": "Melany Matias", "age": 10, "address": { "number": 8838, "street": "Main St.", "city": "Seattle" }, "interests": [ "Coffee", "Tennis", "Bass" ], "children": [ { "name": "Earnestine Matias", "age": null }, { "name": "Lore Matias", "age": null } ] }
+{ "cid": 379, "name": "Penney Huslander", "age": 58, "address": { "number": 6919, "street": "7th St.", "city": "Portland" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Magaret Huslander", "age": null }, { "name": "Dodie Huslander", "age": 14 } ] }
+{ "cid": 380, "name": "Silva Purdue", "age": 33, "address": { "number": 1759, "street": "7th St.", "city": "Portland" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Marshall Purdue", "age": null }, { "name": "Yuki Purdue", "age": null }, { "name": "Val Purdue", "age": 12 }, { "name": "Dominica Purdue", "age": null } ] }
+{ "cid": 381, "name": "Kassandra Ereth", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Databases", "Walking" ], "children": [ { "name": "Angelina Ereth", "age": 46 }, { "name": "Tristan Ereth", "age": null }, { "name": "Johnny Ereth", "age": null } ] }
+{ "cid": 383, "name": "Marty Castine", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nakisha Castine", "age": 40 }, { "name": "Mina Castine", "age": null }, { "name": "Katrice Castine", "age": 56 }, { "name": "Reuben Castine", "age": null } ] }
+{ "cid": 385, "name": "Jody Favaron", "age": 73, "address": { "number": 4724, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [ { "name": "Elane Favaron", "age": 47 }, { "name": "Katherine Favaron", "age": 38 } ] }
+{ "cid": 386, "name": "Mao Gradowski", "age": 36, "address": { "number": 5116, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Jeneva Gradowski", "age": null }, { "name": "Thu Gradowski", "age": 22 }, { "name": "Daphine Gradowski", "age": null }, { "name": "Providencia Gradowski", "age": null } ] }
+{ "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 390, "name": "Shera Cung", "age": 69, "address": { "number": 5850, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Fishing", "Computers", "Cigars", "Base Jumping" ], "children": [ { "name": "Lenore Cung", "age": 20 } ] }
+{ "cid": 391, "name": "Lynn Gregory", "age": 51, "address": { "number": 1249, "street": "Hill St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Jeannine Gregory", "age": null }, { "name": "Jaymie Gregory", "age": null }, { "name": "Lorrine Gregory", "age": 37 } ] }
+{ "cid": 392, "name": "Isiah Nussbaumer", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 393, "name": "Rossana Monton", "age": 34, "address": { "number": 4490, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Base Jumping" ], "children": [ { "name": "Glayds Monton", "age": null }, { "name": "Lily Monton", "age": null }, { "name": "Raina Monton", "age": null }, { "name": "Hilma Monton", "age": null } ] }
+{ "cid": 394, "name": "Lizette Roux", "age": 57, "address": { "number": 458, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books" ], "children": [ { "name": "Doloris Roux", "age": null } ] }
+{ "cid": 395, "name": "Bob Layman", "age": 61, "address": { "number": 3646, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 396, "name": "Delfina Calcara", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Sybil Calcara", "age": null } ] }
+{ "cid": 397, "name": "Blake Kealy", "age": 34, "address": { "number": 2156, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Databases", "Wine", "Cigars" ], "children": [ { "name": "Lorenza Kealy", "age": null }, { "name": "Beula Kealy", "age": 15 }, { "name": "Kristofer Kealy", "age": null }, { "name": "Shayne Kealy", "age": null } ] }
+{ "cid": 398, "name": "Piedad Paranada", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Claribel Paranada", "age": 22 }, { "name": "Lincoln Paranada", "age": null }, { "name": "Cecilia Paranada", "age": null } ] }
+{ "cid": 399, "name": "Myra Millwee", "age": null, "address": null, "interests": [ "Tennis", "Running", "Tennis" ], "children": [ { "name": "Gaye Millwee", "age": null } ] }
+{ "cid": 400, "name": "Jeffery Maresco", "age": null, "address": null, "interests": [ "Coffee", "Bass" ], "children": [  ] }
+{ "cid": 401, "name": "Moises Jago", "age": 27, "address": { "number": 3773, "street": "Main St.", "city": "San Jose" }, "interests": [ "Music" ], "children": [ { "name": "Shoshana Jago", "age": null }, { "name": "Juliet Jago", "age": null }, { "name": "Berneice Jago", "age": 13 } ] }
+{ "cid": 402, "name": "Terrilyn Shinall", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Music" ], "children": [ { "name": "Minh Shinall", "age": null }, { "name": "Diedre Shinall", "age": 22 } ] }
+{ "cid": 403, "name": "Kayleigh Houey", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Ta Houey", "age": null }, { "name": "Ayana Houey", "age": null }, { "name": "Dominique Houey", "age": null }, { "name": "Denise Houey", "age": 48 } ] }
+{ "cid": 404, "name": "Harriette Abo", "age": null, "address": null, "interests": [ "Walking", "Running" ], "children": [  ] }
+{ "cid": 405, "name": "Shawnda Landborg", "age": 73, "address": { "number": 2396, "street": "Hill St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Cherrie Landborg", "age": 10 } ] }
+{ "cid": 406, "name": "Addie Mandez", "age": null, "address": null, "interests": [ "Tennis", "Cigars", "Books" ], "children": [ { "name": "Rosendo Mandez", "age": 34 } ] }
+{ "cid": 407, "name": "Bebe Cotney", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Daren Cotney", "age": null }, { "name": "Lady Cotney", "age": 48 } ] }
+{ "cid": 408, "name": "Ava Zornes", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 410, "name": "Jennie Longhenry", "age": 82, "address": { "number": 7427, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Charles Longhenry", "age": 61 }, { "name": "Faviola Longhenry", "age": 25 }, { "name": "Darline Longhenry", "age": null }, { "name": "Lorean Longhenry", "age": null } ] }
+{ "cid": 411, "name": "Cindi Pepin", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Fallon Pepin", "age": 39 }, { "name": "Armanda Pepin", "age": null }, { "name": "Loriann Pepin", "age": null }, { "name": "Bambi Pepin", "age": 43 } ] }
+{ "cid": 412, "name": "Devon Szalai", "age": 26, "address": { "number": 2384, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books", "Books" ], "children": [ { "name": "Yolonda Szalai", "age": null }, { "name": "Denita Szalai", "age": null }, { "name": "Priscila Szalai", "age": 10 }, { "name": "Cassondra Szalai", "age": 12 } ] }
+{ "cid": 413, "name": "Maurice Landrie", "age": null, "address": null, "interests": [ "Computers", "Coffee" ], "children": [ { "name": "Gail Landrie", "age": 37 }, { "name": "Carylon Landrie", "age": null }, { "name": "Allen Landrie", "age": 16 }, { "name": "Andreas Landrie", "age": null } ] }
+{ "cid": 414, "name": "Sixta Smithheart", "age": null, "address": null, "interests": [ "Skiing", "Books", "Computers" ], "children": [ { "name": "Nicholas Smithheart", "age": null } ] }
+{ "cid": 415, "name": "Valentin Mclarney", "age": null, "address": null, "interests": [ "Squash", "Squash", "Video Games" ], "children": [ { "name": "Vanda Mclarney", "age": 17 } ] }
+{ "cid": 417, "name": "Irene Funderberg", "age": 45, "address": { "number": 8503, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Skiing", "Running" ], "children": [ { "name": "Lyndia Funderberg", "age": 14 }, { "name": "Herta Funderberg", "age": null } ] }
+{ "cid": 418, "name": "Gavin Delpino", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Gianna Delpino", "age": null }, { "name": "Carmella Delpino", "age": 55 } ] }
+{ "cid": 419, "name": "Hector Brisbone", "age": null, "address": null, "interests": [ "Databases", "Books", "Walking", "Databases" ], "children": [ { "name": "Frederick Brisbone", "age": 17 } ] }
+{ "cid": 420, "name": "Coralie Regueira", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Latoyia Regueira", "age": 31 }, { "name": "Obdulia Regueira", "age": 12 }, { "name": "Herlinda Regueira", "age": null } ] }
+{ "cid": 421, "name": "Rubye Dillabough", "age": 55, "address": { "number": 6980, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Squash" ], "children": [ { "name": "Hyacinth Dillabough", "age": 19 }, { "name": "Arie Dillabough", "age": null } ] }
+{ "cid": 422, "name": "Annmarie Whitcher", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Honey Whitcher", "age": null }, { "name": "Dan Whitcher", "age": 22 } ] }
+{ "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] }
+{ "cid": 426, "name": "Agripina Philley", "age": 79, "address": { "number": 1533, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Georgianne Philley", "age": null }, { "name": "Neville Philley", "age": null }, { "name": "Brande Philley", "age": 42 }, { "name": "Tanisha Philley", "age": null } ] }
+{ "cid": 427, "name": "Janay Presutti", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Julietta Presutti", "age": null } ] }
+{ "cid": 428, "name": "Tiffany Waye", "age": null, "address": null, "interests": [ "Basketball", "Cigars" ], "children": [ { "name": "Berna Waye", "age": null }, { "name": "Kiersten Waye", "age": null }, { "name": "Romeo Waye", "age": null }, { "name": "Marvel Waye", "age": 56 } ] }
+{ "cid": 429, "name": "Eladia Scannell", "age": 20, "address": { "number": 5036, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Music", "Movies" ], "children": [  ] }
+{ "cid": 430, "name": "Cari Woll", "age": 45, "address": { "number": 8226, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Walking", "Cooking" ], "children": [ { "name": "Tomasa Woll", "age": 32 }, { "name": "Annika Woll", "age": 21 } ] }
+{ "cid": 431, "name": "Estela Tolbent", "age": 27, "address": { "number": 7186, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Joie Tolbent", "age": null }, { "name": "Angila Tolbent", "age": null }, { "name": "Anastasia Tolbent", "age": 14 } ] }
+{ "cid": 432, "name": "Judi Vinet", "age": 85, "address": { "number": 7304, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Golden Vinet", "age": 20 }, { "name": "Maragret Vinet", "age": null }, { "name": "Keshia Vinet", "age": 10 }, { "name": "Gary Vinet", "age": 73 } ] }
+{ "cid": 433, "name": "Caleb Merrbach", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Amado Merrbach", "age": 45 } ] }
+{ "cid": 434, "name": "Tamesha Soho", "age": 33, "address": { "number": 4534, "street": "Park St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Cody Soho", "age": null }, { "name": "Glennie Soho", "age": 22 } ] }
+{ "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }
+{ "cid": 436, "name": "Xenia Pool", "age": null, "address": null, "interests": [ "Books" ], "children": [  ] }
+{ "cid": 437, "name": "Marlene Macintyre", "age": 86, "address": { "number": 3708, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Music", "Coffee" ], "children": [ { "name": "Todd Macintyre", "age": null }, { "name": "Mechelle Macintyre", "age": 50 } ] }
+{ "cid": 438, "name": "Allegra Pefanis", "age": null, "address": null, "interests": [ "Computers", "Music", "Cigars" ], "children": [  ] }
+{ "cid": 439, "name": "Lillia Villnave", "age": 34, "address": { "number": 9212, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Otis Villnave", "age": null } ] }
+{ "cid": 440, "name": "Rosie Shappen", "age": null, "address": null, "interests": [ "Cooking", "Music", "Cigars" ], "children": [ { "name": "Jung Shappen", "age": 11 } ] }
+{ "cid": 441, "name": "Jamison Reeser", "age": 84, "address": { "number": 9376, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [ { "name": "Elena Reeser", "age": 28 } ] }
+{ "cid": 442, "name": "Val Disorda", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Simone Disorda", "age": 53 }, { "name": "Jacalyn Disorda", "age": 41 }, { "name": "Ron Disorda", "age": null }, { "name": "Clifton Disorda", "age": null } ] }
+{ "cid": 445, "name": "Walton Komo", "age": 16, "address": { "number": 8769, "street": "Main St.", "city": "Seattle" }, "interests": [ "Running", "Basketball", "Tennis" ], "children": [  ] }
+{ "cid": 446, "name": "Lilly Grannell", "age": 21, "address": { "number": 5894, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Computers", "Tennis", "Puzzles", "Books" ], "children": [ { "name": "Victor Grannell", "age": null } ] }
+{ "cid": 447, "name": "Iris Schoneman", "age": 34, "address": { "number": 7648, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Wine", "Puzzles", "Cigars" ], "children": [ { "name": "Shemika Schoneman", "age": 11 }, { "name": "Maritza Schoneman", "age": 21 }, { "name": "Martha Schoneman", "age": 20 } ] }
+{ "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] }
+{ "cid": 449, "name": "Jacinda Markle", "age": null, "address": null, "interests": [ "Basketball", "Basketball", "Computers" ], "children": [ { "name": "Tam Markle", "age": 45 } ] }
+{ "cid": 450, "name": "Althea Mohammed", "age": null, "address": null, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Jasper Mohammed", "age": null } ] }
+{ "cid": 451, "name": "Lelia Sondelski", "age": 60, "address": { "number": 4044, "street": "Park St.", "city": "Portland" }, "interests": [ "Books", "Squash", "Walking" ], "children": [  ] }
+{ "cid": 452, "name": "Casie Marasigan", "age": null, "address": null, "interests": [ "Walking", "Computers" ], "children": [ { "name": "Connie Marasigan", "age": null }, { "name": "Kimberlie Marasigan", "age": null } ] }
+{ "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] }
+{ "cid": 454, "name": "Irving Lhuillier", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Emile Lhuillier", "age": null }, { "name": "Albert Lhuillier", "age": null }, { "name": "Ingeborg Lhuillier", "age": 23 }, { "name": "Shila Lhuillier", "age": 55 } ] }
+{ "cid": 455, "name": "Manual Altizer", "age": 70, "address": { "number": 6293, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Fishing", "Coffee" ], "children": [ { "name": "Katherine Altizer", "age": null } ] }
+{ "cid": 456, "name": "Kim Cervera", "age": 89, "address": { "number": 3967, "street": "Lake St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Winona Cervera", "age": 37 }, { "name": "Shanice Cervera", "age": null }, { "name": "Michaele Cervera", "age": null } ] }
+{ "cid": 457, "name": "Jenice Boger", "age": null, "address": null, "interests": [ "Skiing", "Databases", "Running" ], "children": [  ] }
+{ "cid": 458, "name": "Ivan Sien", "age": 17, "address": { "number": 9981, "street": "Lake St.", "city": "Portland" }, "interests": [ "Cooking", "Coffee" ], "children": [ { "name": "Laurence Sien", "age": null }, { "name": "Nelle Sien", "age": null }, { "name": "Thalia Sien", "age": null } ] }
+{ "cid": 459, "name": "Mable Ellwein", "age": 60, "address": { "number": 1138, "street": "Lake St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Stan Ellwein", "age": 19 }, { "name": "Ashlea Ellwein", "age": 13 }, { "name": "Tiesha Ellwein", "age": 28 } ] }
+{ "cid": 460, "name": "Jeraldine Choules", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Berneice Choules", "age": 16 }, { "name": "Jaime Choules", "age": 21 }, { "name": "Li Choules", "age": 20 }, { "name": "Leah Choules", "age": null } ] }
+{ "cid": 461, "name": "Dessie Schnibbe", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] }
+{ "cid": 463, "name": "Mika Rininger", "age": null, "address": null, "interests": [ "Databases", "Cooking" ], "children": [ { "name": "Inez Rininger", "age": 58 }, { "name": "Betty Rininger", "age": null }, { "name": "Laurie Rininger", "age": 48 }, { "name": "Billie Rininger", "age": null } ] }
+{ "cid": 464, "name": "Petra Kinsel", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Janise Kinsel", "age": null }, { "name": "Donnie Kinsel", "age": 26 }, { "name": "Joana Kinsel", "age": 12 } ] }
+{ "cid": 465, "name": "Rey Arango", "age": 68, "address": { "number": 1788, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 466, "name": "Paulene Bagen", "age": 87, "address": { "number": 4093, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music" ], "children": [ { "name": "Antione Bagen", "age": null }, { "name": "Samatha Bagen", "age": null } ] }
+{ "cid": 467, "name": "Magali Ingerson", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Monty Ingerson", "age": 11 }, { "name": "Noelia Ingerson", "age": 47 }, { "name": "Tennie Ingerson", "age": null }, { "name": "Merrill Ingerson", "age": null } ] }
+{ "cid": 468, "name": "Raeann Conry", "age": 68, "address": { "number": 4312, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Squash" ], "children": [ { "name": "Ellena Conry", "age": 36 }, { "name": "Lynwood Conry", "age": 13 }, { "name": "Coreen Conry", "age": 23 } ] }
+{ "cid": 470, "name": "Yesenia Doyon", "age": 78, "address": { "number": 3641, "street": "7th St.", "city": "Seattle" }, "interests": [ "Databases", "Puzzles" ], "children": [ { "name": "Halley Doyon", "age": null }, { "name": "Teisha Doyon", "age": 33 }, { "name": "Warren Doyon", "age": null } ] }
+{ "cid": 471, "name": "Nicol Majersky", "age": null, "address": null, "interests": [ "Video Games", "Books" ], "children": [ { "name": "Alise Majersky", "age": null }, { "name": "Kathline Majersky", "age": 53 }, { "name": "Charlie Majersky", "age": 45 }, { "name": "Helaine Majersky", "age": null } ] }
+{ "cid": 472, "name": "Kelley Mischler", "age": 38, "address": { "number": 7988, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Movies", "Cooking", "Skiing" ], "children": [ { "name": "Keila Mischler", "age": 19 }, { "name": "Evie Mischler", "age": 15 } ] }
+{ "cid": 475, "name": "Brinda Gouker", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Gayle Gouker", "age": 52 } ] }
+{ "cid": 478, "name": "Sophia Whitt", "age": 26, "address": { "number": 2787, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Irving Whitt", "age": 13 }, { "name": "Jeannette Whitt", "age": null } ] }
+{ "cid": 479, "name": "Danilo Varney", "age": 17, "address": { "number": 9330, "street": "Hill St.", "city": "Portland" }, "interests": [ "Wine" ], "children": [ { "name": "Shelby Varney", "age": null }, { "name": "Fidela Varney", "age": null }, { "name": "Maynard Varney", "age": null }, { "name": "Lindsay Varney", "age": null } ] }
+{ "cid": 480, "name": "Nigel Pitmon", "age": null, "address": null, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Janene Pitmon", "age": null }, { "name": "Louie Pitmon", "age": 19 }, { "name": "Genny Pitmon", "age": 24 }, { "name": "Robby Pitmon", "age": 55 } ] }
+{ "cid": 481, "name": "Leana Revera", "age": null, "address": null, "interests": [ "Running", "Skiing" ], "children": [ { "name": "Marquita Revera", "age": null } ] }
+{ "cid": 482, "name": "Samantha Stonis", "age": null, "address": null, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 483, "name": "Elsa Vigen", "age": null, "address": null, "interests": [ "Wine", "Databases" ], "children": [ { "name": "Larae Vigen", "age": null }, { "name": "Elwood Vigen", "age": null } ] }
+{ "cid": 484, "name": "Bennie Dragaj", "age": null, "address": null, "interests": [ "Fishing", "Databases", "Wine" ], "children": [ { "name": "Viva Dragaj", "age": 13 } ] }
+{ "cid": 485, "name": "Gene Rogoff", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Ebonie Rogoff", "age": null } ] }
+{ "cid": 486, "name": "Willa Patman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ross Patman", "age": 42 }, { "name": "Erin Patman", "age": null }, { "name": "Vannessa Patman", "age": 11 }, { "name": "Hilaria Patman", "age": 28 } ] }
+{ "cid": 487, "name": "Zenia Virgilio", "age": 46, "address": { "number": 584, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Walking", "Squash", "Wine" ], "children": [ { "name": "Quintin Virgilio", "age": null }, { "name": "Edith Virgilio", "age": null }, { "name": "Nicolle Virgilio", "age": 33 } ] }
+{ "cid": 489, "name": "Brigid Delosier", "age": 31, "address": { "number": 6082, "street": "Oak St.", "city": "Portland" }, "interests": [ "Tennis", "Cigars", "Music" ], "children": [ { "name": "Allegra Delosier", "age": null }, { "name": "Yong Delosier", "age": 10 }, { "name": "Steffanie Delosier", "age": 13 } ] }
+{ "cid": 492, "name": "Gene Alcazar", "age": 59, "address": { "number": 9650, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Computers" ], "children": [ { "name": "Olympia Alcazar", "age": null }, { "name": "Mark Alcazar", "age": 37 }, { "name": "Danilo Alcazar", "age": null } ] }
+{ "cid": 493, "name": "Lindsey Trout", "age": 86, "address": { "number": 7619, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Madlyn Trout", "age": 58 }, { "name": "Amie Trout", "age": 72 } ] }
+{ "cid": 494, "name": "Delma Deever", "age": 84, "address": { "number": 5044, "street": "7th St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Squash" ], "children": [  ] }
+{ "cid": 496, "name": "Lonna Starkweather", "age": 80, "address": { "number": 1162, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Bass", "Running" ], "children": [ { "name": "Matilda Starkweather", "age": null } ] }
+{ "cid": 497, "name": "Chantay Balak", "age": null, "address": null, "interests": [ "Bass", "Fishing" ], "children": [ { "name": "John Balak", "age": null }, { "name": "Thu Balak", "age": 38 } ] }
+{ "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }
+{ "cid": 499, "name": "Carlita Tarlton", "age": 43, "address": { "number": 9148, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Computers", "Base Jumping", "Video Games" ], "children": [  ] }
+{ "cid": 500, "name": "Tierra Bjorklund", "age": null, "address": null, "interests": [ "Puzzles", "Skiing" ], "children": [ { "name": "Avelina Bjorklund", "age": 54 }, { "name": "Mallory Bjorklund", "age": null } ] }
+{ "cid": 501, "name": "Alyce Coant", "age": null, "address": null, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Elyse Coant", "age": 50 } ] }
+{ "cid": 502, "name": "Lawana Mulik", "age": 82, "address": { "number": 3071, "street": "Park St.", "city": "Portland" }, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Carrie Mulik", "age": null }, { "name": "Sharlene Mulik", "age": 33 }, { "name": "Leone Mulik", "age": 46 } ] }
+{ "cid": 503, "name": "Phyliss Cassani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Rolando Cassani", "age": 44 }, { "name": "Rikki Cassani", "age": 18 }, { "name": "Monty Cassani", "age": 40 } ] }
+{ "cid": 504, "name": "Marla Kolenda", "age": 57, "address": { "number": 464, "street": "View St.", "city": "San Jose" }, "interests": [ "Coffee" ], "children": [ { "name": "Iliana Kolenda", "age": 34 }, { "name": "Ammie Kolenda", "age": 20 }, { "name": "Candi Kolenda", "age": 23 }, { "name": "Lyla Kolenda", "age": 23 } ] }
+{ "cid": 505, "name": "Mike Runk", "age": null, "address": null, "interests": [ "Databases", "Computers", "Running", "Video Games" ], "children": [ { "name": "Lashawn Runk", "age": 21 } ] }
+{ "cid": 506, "name": "Jonna Kolbusz", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Debrah Kolbusz", "age": null }, { "name": "Hugh Kolbusz", "age": null } ] }
+{ "cid": 507, "name": "Yuk Flanegan", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Squash" ], "children": [ { "name": "Alexander Flanegan", "age": null } ] }
+{ "cid": 508, "name": "Tiffany Kimmey", "age": 64, "address": { "number": 8625, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Bass", "Walking" ], "children": [  ] }
+{ "cid": 509, "name": "Alvaro Johnke", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Allison Johnke", "age": null }, { "name": "Ellan Johnke", "age": null } ] }
+{ "cid": 510, "name": "Candace Morello", "age": null, "address": null, "interests": [ "Wine", "Base Jumping", "Running" ], "children": [ { "name": "Sandy Morello", "age": 57 }, { "name": "Delois Morello", "age": 15 } ] }
+{ "cid": 512, "name": "Paul Cobian", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Will Cobian", "age": 30 }, { "name": "Conrad Cobian", "age": 35 }, { "name": "Justin Cobian", "age": 11 } ] }
+{ "cid": 513, "name": "Marianna Gortman", "age": 49, "address": { "number": 927, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Databases" ], "children": [  ] }
+{ "cid": 514, "name": "Raleigh Belling", "age": 56, "address": { "number": 7408, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [  ] }
+{ "cid": 515, "name": "Connie Banis", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittni Banis", "age": null }, { "name": "Deloras Banis", "age": 25 } ] }
+{ "cid": 516, "name": "Taunya Berkbigler", "age": 82, "address": { "number": 5441, "street": "View St.", "city": "Seattle" }, "interests": [ "Databases", "Tennis" ], "children": [ { "name": "Cherry Berkbigler", "age": 27 }, { "name": "Perry Berkbigler", "age": null } ] }
+{ "cid": 517, "name": "Alfonso Bruderer", "age": null, "address": null, "interests": [ "Bass" ], "children": [  ] }
+{ "cid": 518, "name": "Cora Ingargiola", "age": null, "address": null, "interests": [ "Skiing", "Squash", "Movies" ], "children": [ { "name": "Katlyn Ingargiola", "age": null }, { "name": "Mike Ingargiola", "age": null }, { "name": "Lawrence Ingargiola", "age": null }, { "name": "Isabelle Ingargiola", "age": null } ] }
+{ "cid": 519, "name": "Julianna Goodsell", "age": 59, "address": { "number": 5594, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Video Games", "Fishing" ], "children": [  ] }
+{ "cid": 520, "name": "Janay Bernbeck", "age": null, "address": null, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Aurea Bernbeck", "age": null }, { "name": "Tiara Bernbeck", "age": null }, { "name": "Alfredia Bernbeck", "age": 26 } ] }
+{ "cid": 521, "name": "Frankie Hofmann", "age": null, "address": null, "interests": [ "Databases", "Movies" ], "children": [ { "name": "Shirlee Hofmann", "age": 32 }, { "name": "Jacque Hofmann", "age": 23 }, { "name": "Jazmin Hofmann", "age": null }, { "name": "Serena Hofmann", "age": 56 } ] }
+{ "cid": 522, "name": "Daryl Kissack", "age": 86, "address": { "number": 7825, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Squash", "Base Jumping", "Tennis" ], "children": [ { "name": "Darrel Kissack", "age": 21 } ] }
+{ "cid": 523, "name": "Johanne Huls", "age": null, "address": null, "interests": [ "Books", "Bass" ], "children": [ { "name": "Melynda Huls", "age": null }, { "name": "Vicky Huls", "age": 16 }, { "name": "Charlott Huls", "age": null } ] }
+{ "cid": 524, "name": "Rickie Manche", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 525, "name": "Miquel Hodnefield", "age": 12, "address": { "number": 4784, "street": "7th St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Darnell Hodnefield", "age": null }, { "name": "Particia Hodnefield", "age": null } ] }
+{ "cid": 528, "name": "Tamela Witherbee", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Penney Witherbee", "age": null } ] }
+{ "cid": 529, "name": "Cinderella Lewis", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Flor Lewis", "age": null }, { "name": "Alonzo Lewis", "age": 23 } ] }
+{ "cid": 530, "name": "Olevia Sturk", "age": 72, "address": { "number": 1939, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Cindy Sturk", "age": 18 }, { "name": "Alishia Sturk", "age": null }, { "name": "Sonja Sturk", "age": 51 } ] }
+{ "cid": 531, "name": "Camelia Yoes", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 532, "name": "Tania Fraklin", "age": 38, "address": { "number": 2857, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Squash", "Databases" ], "children": [  ] }
+{ "cid": 533, "name": "Trinity Urquidez", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Corrine Urquidez", "age": 29 }, { "name": "Markita Urquidez", "age": 19 }, { "name": "Danette Urquidez", "age": null } ] }
+{ "cid": 534, "name": "Bridgett Ebel", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 535, "name": "Juana Hirliman", "age": 87, "address": { "number": 6763, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Ursula Hirliman", "age": 40 }, { "name": "Doretha Hirliman", "age": 30 }, { "name": "Leisha Hirliman", "age": 49 } ] }
+{ "cid": 536, "name": "Wilber Rehrer", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Zulema Rehrer", "age": null }, { "name": "Lavonda Rehrer", "age": null }, { "name": "Stacey Rehrer", "age": 59 } ] }
+{ "cid": 537, "name": "Mara Hugar", "age": null, "address": null, "interests": [ "Fishing", "Skiing", "Skiing" ], "children": [ { "name": "Krista Hugar", "age": null } ] }
+{ "cid": 538, "name": "Mack Vollick", "age": null, "address": null, "interests": [ "Base Jumping", "Fishing", "Walking", "Computers" ], "children": [ { "name": "Gil Vollick", "age": 11 }, { "name": "Marica Vollick", "age": null } ] }
+{ "cid": 539, "name": "Nicky Graceffo", "age": null, "address": null, "interests": [ "Video Games" ], "children": [  ] }
+{ "cid": 540, "name": "Bryanna Herling", "age": 67, "address": { "number": 7682, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Cyrstal Herling", "age": 50 }, { "name": "Vallie Herling", "age": 54 }, { "name": "Doris Herling", "age": null } ] }
+{ "cid": 541, "name": "Sammy Adamitis", "age": 71, "address": { "number": 5593, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Books", "Tennis", "Cooking" ], "children": [  ] }
+{ "cid": 542, "name": "Eveline Smedley", "age": 50, "address": { "number": 5513, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Lynsey Smedley", "age": 26 } ] }
+{ "cid": 543, "name": "Pearl Nollette", "age": null, "address": null, "interests": [ "Base Jumping", "Running" ], "children": [  ] }
+{ "cid": 544, "name": "Silas Demay", "age": 69, "address": { "number": 447, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Bass" ], "children": [ { "name": "Latonya Demay", "age": null }, { "name": "Lissette Demay", "age": 37 }, { "name": "Lynell Demay", "age": 42 }, { "name": "Mikel Demay", "age": 17 } ] }
+{ "cid": 545, "name": "Dolores Ferer", "age": null, "address": null, "interests": [ "Coffee", "Bass", "Tennis" ], "children": [ { "name": "Bridgette Ferer", "age": null } ] }
+{ "cid": 547, "name": "Daryl Dambra", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Jacquline Dambra", "age": null }, { "name": "Seymour Dambra", "age": null } ] }
+{ "cid": 548, "name": "Elvia Duchesney", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Arcelia Duchesney", "age": 22 } ] }
+{ "cid": 549, "name": "Kathrin Cruff", "age": 63, "address": { "number": 9002, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Tennis", "Books" ], "children": [ { "name": "Candi Cruff", "age": 49 }, { "name": "Barry Cruff", "age": 17 }, { "name": "Shane Cruff", "age": 18 }, { "name": "Brendon Cruff", "age": null } ] }
+{ "cid": 550, "name": "Aleisha Brehon", "age": 61, "address": { "number": 7835, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Squash" ], "children": [ { "name": "Vito Brehon", "age": null }, { "name": "Matthew Brehon", "age": 32 } ] }
+{ "cid": 552, "name": "Marlena Humann", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 553, "name": "Mina Ciminera", "age": null, "address": null, "interests": [ "Base Jumping", "Databases" ], "children": [ { "name": "Cornelius Ciminera", "age": null }, { "name": "Rozanne Ciminera", "age": null }, { "name": "Byron Ciminera", "age": null } ] }
+{ "cid": 554, "name": "Darci Yafai", "age": 60, "address": { "number": 4694, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Lecia Yafai", "age": 47 } ] }
+{ "cid": 555, "name": "Agustina Bretthauer", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Arthur Bretthauer", "age": 33 }, { "name": "Titus Bretthauer", "age": 33 }, { "name": "Margret Bretthauer", "age": null } ] }
+{ "cid": 557, "name": "Kaitlyn Hilleman", "age": 61, "address": { "number": 1076, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Corrie Hilleman", "age": 31 }, { "name": "Jovan Hilleman", "age": null }, { "name": "Carmine Hilleman", "age": null } ] }
+{ "cid": 559, "name": "Carolyne Shiroma", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Ying Shiroma", "age": 57 } ] }
+{ "cid": 560, "name": "Karin Dicesare", "age": null, "address": null, "interests": [ "Wine", "Puzzles" ], "children": [  ] }
+{ "cid": 561, "name": "Renetta Cudworth", "age": null, "address": null, "interests": [ "Skiing", "Basketball" ], "children": [  ] }
+{ "cid": 563, "name": "Deirdre Landero", "age": null, "address": null, "interests": [ "Books", "Fishing", "Video Games" ], "children": [ { "name": "Norman Landero", "age": 59 }, { "name": "Jennine Landero", "age": 45 }, { "name": "Rutha Landero", "age": 19 }, { "name": "Jackie Landero", "age": 29 } ] }
+{ "cid": 564, "name": "Inger Dargin", "age": 56, "address": { "number": 8704, "street": "View St.", "city": "Mountain View" }, "interests": [ "Wine", "Running", "Computers" ], "children": [  ] }
+{ "cid": 565, "name": "Shantell Rima", "age": 82, "address": { "number": 205, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Boyce Rima", "age": 67 }, { "name": "Woodrow Rima", "age": 18 }, { "name": "Helene Rima", "age": null }, { "name": "David Rima", "age": null } ] }
+{ "cid": 566, "name": "Asley Grow", "age": null, "address": null, "interests": [ "Coffee", "Books", "Tennis" ], "children": [ { "name": "Dale Grow", "age": null } ] }
+{ "cid": 567, "name": "Peggie Madhavan", "age": null, "address": null, "interests": [ "Computers", "Bass" ], "children": [  ] }
+{ "cid": 569, "name": "Beata Diles", "age": 88, "address": { "number": 2198, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Myrtice Diles", "age": 46 }, { "name": "Stella Diles", "age": null }, { "name": "Rowena Diles", "age": 26 } ] }
+{ "cid": 570, "name": "Lee Basora", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [  ] }
+{ "cid": 571, "name": "Lenita Tentler", "age": null, "address": null, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Damian Tentler", "age": 16 }, { "name": "Camellia Tentler", "age": null }, { "name": "Vern Tentler", "age": 15 } ] }
+{ "cid": 572, "name": "Darcy Polycarpe", "age": 35, "address": { "number": 8051, "street": "View St.", "city": "Mountain View" }, "interests": [ "Computers", "Coffee", "Walking", "Walking" ], "children": [ { "name": "Kenneth Polycarpe", "age": null } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 574, "name": "Camellia Toxey", "age": 52, "address": { "number": 5437, "street": "Hill St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Deandrea Toxey", "age": null }, { "name": "Danille Toxey", "age": null } ] }
+{ "cid": 577, "name": "Alejandro Oblinger", "age": null, "address": null, "interests": [ "Movies", "Movies" ], "children": [ { "name": "Tenesha Oblinger", "age": 56 }, { "name": "Loni Oblinger", "age": 12 }, { "name": "Sherryl Oblinger", "age": null } ] }
+{ "cid": 578, "name": "Dolly Delphia", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Sharron Delphia", "age": null }, { "name": "Shemeka Delphia", "age": null }, { "name": "Rachael Delphia", "age": null } ] }
+{ "cid": 579, "name": "Sabra Yuenger", "age": 45, "address": { "number": 2681, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Eddie Yuenger", "age": null } ] }
+{ "cid": 581, "name": "Leigha Finkenbinder", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lorine Finkenbinder", "age": 29 }, { "name": "Stephanie Finkenbinder", "age": 28 } ] }
+{ "cid": 582, "name": "Suzie Ocallahan", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Tamra Ocallahan", "age": null } ] }
+{ "cid": 583, "name": "Bev Yerena", "age": null, "address": null, "interests": [ "Puzzles", "Wine" ], "children": [ { "name": "Larhonda Yerena", "age": 45 }, { "name": "Josefina Yerena", "age": null }, { "name": "Sydney Yerena", "age": 42 } ] }
+{ "cid": 584, "name": "Bailey Janes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marylou Janes", "age": null }, { "name": "Andra Janes", "age": null } ] }
+{ "cid": 585, "name": "Young Drube", "age": 21, "address": { "number": 6960, "street": "View St.", "city": "Seattle" }, "interests": [ "Basketball", "Fishing", "Walking" ], "children": [ { "name": "Irwin Drube", "age": null }, { "name": "Gustavo Drube", "age": null } ] }
+{ "cid": 586, "name": "Jeannine Donnerberg", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mike Donnerberg", "age": null } ] }
+{ "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] }
+{ "cid": 588, "name": "Debora Laughinghouse", "age": 87, "address": { "number": 5099, "street": "View St.", "city": "San Jose" }, "interests": [ "Tennis", "Walking", "Databases" ], "children": [ { "name": "Frederica Laughinghouse", "age": 59 }, { "name": "Johnie Laughinghouse", "age": 12 }, { "name": "Numbers Laughinghouse", "age": 73 } ] }
+{ "cid": 589, "name": "Rebeca Blackwell", "age": 66, "address": { "number": 5708, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 590, "name": "Joye Burton", "age": null, "address": null, "interests": [ "Bass", "Base Jumping" ], "children": [ { "name": "Noemi Burton", "age": 19 }, { "name": "Hulda Burton", "age": null }, { "name": "Cleotilde Burton", "age": null }, { "name": "Dara Burton", "age": null } ] }
+{ "cid": 591, "name": "Matthew Tenhaeff", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Jan Tenhaeff", "age": 25 }, { "name": "Nana Tenhaeff", "age": null }, { "name": "Laticia Tenhaeff", "age": null }, { "name": "Ara Tenhaeff", "age": 44 } ] }
+{ "cid": 592, "name": "Rachelle Spare", "age": 13, "address": { "number": 8088, "street": "Oak St.", "city": "Portland" }, "interests": [ "Squash", "Puzzles" ], "children": [ { "name": "Theo Spare", "age": null }, { "name": "Shizue Spare", "age": null } ] }
+{ "cid": 593, "name": "Danial Pittillo", "age": 87, "address": { "number": 815, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Base Jumping" ], "children": [ { "name": "Neva Pittillo", "age": 28 }, { "name": "Brooks Pittillo", "age": null }, { "name": "Randell Pittillo", "age": 52 }, { "name": "Allyson Pittillo", "age": 51 } ] }
+{ "cid": 594, "name": "Zenia Corban", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Video Games", "Cigars" ], "children": [ { "name": "Arielle Corban", "age": null }, { "name": "Arthur Corban", "age": 15 }, { "name": "Taneka Corban", "age": 51 }, { "name": "Claire Corban", "age": null } ] }
+{ "cid": 595, "name": "Samuel Brawdy", "age": 28, "address": { "number": 453, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Books", "Basketball" ], "children": [ { "name": "Marlen Brawdy", "age": 14 }, { "name": "Lorine Brawdy", "age": 13 }, { "name": "Brad Brawdy", "age": null } ] }
+{ "cid": 596, "name": "Juliane Maddy", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Walking", "Basketball" ], "children": [ { "name": "Joannie Maddy", "age": null }, { "name": "Penny Maddy", "age": 35 }, { "name": "Joette Maddy", "age": 35 }, { "name": "Karla Maddy", "age": 54 } ] }
+{ "cid": 597, "name": "Clarine Eutsey", "age": 39, "address": { "number": 9112, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Cigars", "Walking" ], "children": [  ] }
+{ "cid": 598, "name": "Venus Peat", "age": null, "address": null, "interests": [ "Coffee", "Walking", "Cigars" ], "children": [ { "name": "Antonetta Peat", "age": null }, { "name": "Shane Peat", "age": null } ] }
+{ "cid": 599, "name": "Alva Molaison", "age": 87, "address": { "number": 5974, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Milo Molaison", "age": 39 } ] }
+{ "cid": 600, "name": "Cordell Sherburn", "age": null, "address": null, "interests": [ "Squash", "Skiing", "Skiing" ], "children": [ { "name": "Shenna Sherburn", "age": 22 }, { "name": "Minna Sherburn", "age": 10 }, { "name": "Tari Sherburn", "age": null } ] }
+{ "cid": 601, "name": "Zackary Willier", "age": null, "address": null, "interests": [ "Cooking", "Databases", "Databases" ], "children": [  ] }
+{ "cid": 602, "name": "Clyde Salada", "age": 59, "address": { "number": 8316, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Movies", "Skiing", "Cooking" ], "children": [  ] }
+{ "cid": 603, "name": "Barry Corkum", "age": null, "address": null, "interests": [ "Running", "Running" ], "children": [ { "name": "Charlesetta Corkum", "age": null }, { "name": "Helaine Corkum", "age": null }, { "name": "Erinn Corkum", "age": 28 }, { "name": "Alesia Corkum", "age": 36 } ] }
+{ "cid": 605, "name": "Sue Henriksen", "age": 78, "address": { "number": 7208, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Lauretta Henriksen", "age": null }, { "name": "Leigh Henriksen", "age": 11 } ] }
+{ "cid": 606, "name": "Virgilio Liebelt", "age": 11, "address": { "number": 8348, "street": "Cedar St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Stanford Liebelt", "age": null }, { "name": "Delaine Liebelt", "age": null }, { "name": "Kevin Liebelt", "age": null }, { "name": "Michaele Liebelt", "age": null } ] }
+{ "cid": 607, "name": "Bert Garigliano", "age": 71, "address": { "number": 3881, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Walking", "Wine" ], "children": [ { "name": "Junior Garigliano", "age": 42 }, { "name": "Willa Garigliano", "age": 21 }, { "name": "Carlo Garigliano", "age": null } ] }
+{ "cid": 608, "name": "Bruce Stanley", "age": 39, "address": { "number": 4532, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 609, "name": "Mindi Dieudonne", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 610, "name": "Elinor Notoma", "age": 66, "address": { "number": 6763, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Dennis Notoma", "age": null }, { "name": "Carol Notoma", "age": 21 } ] }
+{ "cid": 611, "name": "Evelyne Bassette", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Angla Bassette", "age": 13 } ] }
+{ "cid": 612, "name": "Keneth Ganie", "age": 57, "address": { "number": 7712, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cigars", "Base Jumping" ], "children": [ { "name": "Connie Ganie", "age": null }, { "name": "Kamala Ganie", "age": 25 }, { "name": "Beulah Ganie", "age": 15 } ] }
+{ "cid": 613, "name": "Shanelle Leader", "age": null, "address": null, "interests": [ "Databases", "Base Jumping", "Wine", "Fishing" ], "children": [ { "name": "Florencia Leader", "age": null }, { "name": "Herbert Leader", "age": 11 }, { "name": "Jeanna Leader", "age": null } ] }
+{ "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] }
+{ "cid": 615, "name": "Kimber Warnberg", "age": 77, "address": { "number": 1404, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Kristal Warnberg", "age": null } ] }
+{ "cid": 616, "name": "Shanda Dussault", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Darrick Dussault", "age": null } ] }
+{ "cid": 617, "name": "Jacques Gaskill", "age": null, "address": null, "interests": [ "Cigars", "Coffee", "Computers", "Wine" ], "children": [ { "name": "Angelyn Gaskill", "age": null }, { "name": "Jeanett Gaskill", "age": 40 }, { "name": "Emelda Gaskill", "age": 34 } ] }
+{ "cid": 618, "name": "Janella Hurtt", "age": null, "address": null, "interests": [ "Skiing", "Coffee", "Skiing" ], "children": [ { "name": "Lupe Hurtt", "age": 17 }, { "name": "Jae Hurtt", "age": 14 }, { "name": "Evan Hurtt", "age": 45 } ] }
+{ "cid": 619, "name": "Luanne Elmquist", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Burton Elmquist", "age": 11 }, { "name": "Melvin Elmquist", "age": null } ] }
+{ "cid": 620, "name": "Arielle Mackellar", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Evelin Mackellar", "age": 17 }, { "name": "Theresa Mackellar", "age": 53 }, { "name": "Ronnie Mackellar", "age": null }, { "name": "Elwanda Mackellar", "age": 54 } ] }
+{ "cid": 621, "name": "Theresa Satterthwaite", "age": 16, "address": { "number": 3249, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Rickie Satterthwaite", "age": null }, { "name": "Rina Satterthwaite", "age": null } ] }
+{ "cid": 622, "name": "Telma Rives", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Maribeth Rives", "age": 42 }, { "name": "Youlanda Rives", "age": 13 }, { "name": "Trang Rives", "age": null }, { "name": "Hyun Rives", "age": null } ] }
+{ "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] }
+{ "cid": 625, "name": "Gale Marrazzo", "age": 25, "address": { "number": 2307, "street": "View St.", "city": "San Jose" }, "interests": [ "Fishing", "Base Jumping", "Walking", "Cooking" ], "children": [ { "name": "Coleman Marrazzo", "age": null }, { "name": "Frances Marrazzo", "age": null }, { "name": "Camellia Marrazzo", "age": 11 } ] }
+{ "cid": 626, "name": "Sydney Josten", "age": 44, "address": { "number": 4815, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars" ], "children": [ { "name": "Basil Josten", "age": 14 }, { "name": "Yasuko Josten", "age": null } ] }
+{ "cid": 627, "name": "Fernande Ede", "age": 75, "address": { "number": 9316, "street": "Cedar St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Rebeca Ede", "age": null }, { "name": "Raymond Ede", "age": 57 } ] }
+{ "cid": 628, "name": "Tomoko Alcantara", "age": 56, "address": { "number": 3556, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Tennis" ], "children": [ { "name": "Babara Alcantara", "age": 31 }, { "name": "Ilana Alcantara", "age": null }, { "name": "Maren Alcantara", "age": 45 } ] }
+{ "cid": 629, "name": "Mayola Clabo", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Running" ], "children": [ { "name": "Rigoberto Clabo", "age": 58 } ] }
+{ "cid": 630, "name": "Darla Domenick", "age": 14, "address": { "number": 3315, "street": "Park St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Verda Domenick", "age": null } ] }
+{ "cid": 631, "name": "Brook Jenks", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Eldon Jenks", "age": null }, { "name": "Luann Jenks", "age": 53 }, { "name": "Aurora Jenks", "age": 37 } ] }
+{ "cid": 632, "name": "Keeley Goga", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Walter Goga", "age": 39 }, { "name": "Chaya Goga", "age": null }, { "name": "Melodie Goga", "age": null }, { "name": "Isidro Goga", "age": 32 } ] }
+{ "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] }
+{ "cid": 634, "name": "Katherina Parzych", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Modesta Parzych", "age": null }, { "name": "Darin Parzych", "age": 20 } ] }
+{ "cid": 635, "name": "Angelena Braegelmann", "age": 36, "address": { "number": 4158, "street": "Park St.", "city": "San Jose" }, "interests": [ "Wine", "Skiing" ], "children": [ { "name": "Daisey Braegelmann", "age": 18 }, { "name": "Gaston Braegelmann", "age": 19 }, { "name": "Louella Braegelmann", "age": null }, { "name": "Leonie Braegelmann", "age": null } ] }
+{ "cid": 636, "name": "Babara Shore", "age": 83, "address": { "number": 9452, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Candy Shore", "age": 58 }, { "name": "Nanci Shore", "age": null }, { "name": "Asia Shore", "age": null } ] }
+{ "cid": 639, "name": "Zena Seehusen", "age": 24, "address": { "number": 6303, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Cooking", "Movies", "Music" ], "children": [ { "name": "Hester Seehusen", "age": null }, { "name": "Coreen Seehusen", "age": 12 } ] }
+{ "cid": 640, "name": "Willy Bielak", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 642, "name": "Odell Nova", "age": 25, "address": { "number": 896, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Squash", "Music" ], "children": [ { "name": "Leopoldo Nova", "age": null }, { "name": "Rickey Nova", "age": null }, { "name": "Mike Nova", "age": 14 }, { "name": "Tamie Nova", "age": 14 } ] }
+{ "cid": 643, "name": "Juliet Skreen", "age": null, "address": null, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 644, "name": "Julio Gilly", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Eleonore Gilly", "age": null } ] }
+{ "cid": 645, "name": "Shawnda Dollinger", "age": 36, "address": { "number": 5980, "street": "Park St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Vicente Dollinger", "age": null }, { "name": "Kerrie Dollinger", "age": 10 }, { "name": "Sima Dollinger", "age": 14 } ] }
+{ "cid": 646, "name": "Pablo Catterton", "age": null, "address": null, "interests": [ "Fishing", "Computers" ], "children": [  ] }
+{ "cid": 647, "name": "Jodi Dearson", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [  ] }
+{ "cid": 649, "name": "Anisha Sender", "age": null, "address": null, "interests": [ "Tennis", "Databases", "Bass" ], "children": [ { "name": "Viva Sender", "age": 40 }, { "name": "Terica Sender", "age": null } ] }
+{ "cid": 650, "name": "Darrin Orengo", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Linwood Orengo", "age": 39 } ] }
+{ "cid": 651, "name": "Delana Henk", "age": 69, "address": { "number": 5497, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Video Games", "Databases" ], "children": [ { "name": "Loan Henk", "age": null }, { "name": "Teresa Henk", "age": 20 }, { "name": "Randell Henk", "age": null }, { "name": "Micah Henk", "age": null } ] }
+{ "cid": 652, "name": "Armida Moeuy", "age": 34, "address": { "number": 8306, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running" ], "children": [ { "name": "Sunshine Moeuy", "age": null }, { "name": "Leta Moeuy", "age": 19 } ] }
+{ "cid": 653, "name": "Robbie Rhump", "age": null, "address": null, "interests": [ "Squash", "Computers" ], "children": [ { "name": "Alishia Rhump", "age": 14 }, { "name": "Lyndsay Rhump", "age": 27 } ] }
+{ "cid": 654, "name": "Louis Laubersheimer", "age": 76, "address": { "number": 8010, "street": "7th St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Bass", "Cooking" ], "children": [ { "name": "Jewel Laubersheimer", "age": 22 }, { "name": "Toccara Laubersheimer", "age": 45 }, { "name": "Eve Laubersheimer", "age": null } ] }
+{ "cid": 655, "name": "Shaun Brandenburg", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Base Jumping" ], "children": [ { "name": "Ned Brandenburg", "age": null }, { "name": "Takako Brandenburg", "age": 41 }, { "name": "Astrid Brandenburg", "age": null }, { "name": "Patience Brandenburg", "age": null } ] }
+{ "cid": 656, "name": "Rufus Peaden", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nathanael Peaden", "age": 57 }, { "name": "Jamaal Peaden", "age": null } ] }
+{ "cid": 657, "name": "Rory Teachman", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [  ] }
+{ "cid": 659, "name": "Daniel Groskreutz", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Mariam Groskreutz", "age": 21 }, { "name": "Carlton Groskreutz", "age": null } ] }
+{ "cid": 660, "name": "Israel Aday", "age": null, "address": null, "interests": [ "Wine", "Bass", "Cigars" ], "children": [ { "name": "Mi Aday", "age": null } ] }
+{ "cid": 661, "name": "Lorita Kraut", "age": 43, "address": { "number": 5017, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Movies", "Bass" ], "children": [ { "name": "Mirian Kraut", "age": null } ] }
+{ "cid": 662, "name": "Domonique Corbi", "age": 13, "address": { "number": 7286, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Tennis", "Cooking", "Computers" ], "children": [ { "name": "Katrice Corbi", "age": null }, { "name": "Idalia Corbi", "age": null }, { "name": "Hayley Corbi", "age": null } ] }
+{ "cid": 663, "name": "Riley Noteboom", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marvis Noteboom", "age": 57 } ] }
+{ "cid": 665, "name": "Garnet Desai", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Aliza Desai", "age": null } ] }
+{ "cid": 666, "name": "Pamila Burzlaff", "age": 68, "address": { "number": 6543, "street": "View St.", "city": "Portland" }, "interests": [ "Squash", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 667, "name": "Shaniqua Deist", "age": null, "address": null, "interests": [ "Puzzles", "Books", "Cigars" ], "children": [  ] }
+{ "cid": 668, "name": "Dorene Spigelman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Chiquita Spigelman", "age": 29 }, { "name": "Anisha Spigelman", "age": 34 }, { "name": "Micah Spigelman", "age": 28 } ] }
+{ "cid": 669, "name": "Royal Abke", "age": 60, "address": { "number": 1675, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Leandra Abke", "age": 25 }, { "name": "Shawanna Abke", "age": null } ] }
+{ "cid": 670, "name": "Angelo Kellar", "age": 22, "address": { "number": 3178, "street": "View St.", "city": "Seattle" }, "interests": [ "Wine", "Music", "Fishing" ], "children": [ { "name": "Zula Kellar", "age": null }, { "name": "Brittaney Kellar", "age": 10 }, { "name": "Fredia Kellar", "age": null } ] }
+{ "cid": 671, "name": "Harley Emami", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Valentine Emami", "age": null }, { "name": "Pearlene Emami", "age": null } ] }
+{ "cid": 672, "name": "Pamelia Repka", "age": 30, "address": { "number": 8837, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Coffee", "Base Jumping" ], "children": [ { "name": "Klara Repka", "age": 19 }, { "name": "Bennett Repka", "age": null }, { "name": "Randy Repka", "age": 13 }, { "name": "Ervin Repka", "age": null } ] }
+{ "cid": 673, "name": "Willard Matuszek", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Kyong Matuszek", "age": null }, { "name": "Delena Matuszek", "age": null }, { "name": "Toney Matuszek", "age": null }, { "name": "Shayne Matuszek", "age": 19 } ] }
+{ "cid": 675, "name": "Camellia Brickett", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Leona Brickett", "age": null }, { "name": "Mario Brickett", "age": null }, { "name": "Nadine Brickett", "age": 35 }, { "name": "Marlon Brickett", "age": 31 } ] }
+{ "cid": 676, "name": "Ima Juart", "age": 64, "address": { "number": 2498, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Walking" ], "children": [ { "name": "Cortez Juart", "age": 17 }, { "name": "Guillermo Juart", "age": null }, { "name": "Shelley Juart", "age": 20 }, { "name": "Daryl Juart", "age": null } ] }
+{ "cid": 677, "name": "Brigid Sarabia", "age": 89, "address": { "number": 918, "street": "Park St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Elisa Sarabia", "age": null }, { "name": "Pura Sarabia", "age": 56 } ] }
+{ "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] }
+{ "cid": 680, "name": "Domenica Qunnarath", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 681, "name": "Iliana Nagele", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Sunny Nagele", "age": 55 }, { "name": "Waltraud Nagele", "age": 39 }, { "name": "Darron Nagele", "age": null } ] }
+{ "cid": 682, "name": "Krystle Weingartner", "age": 87, "address": { "number": 5293, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Squash" ], "children": [ { "name": "Bryanna Weingartner", "age": 19 }, { "name": "Rubie Weingartner", "age": 32 }, { "name": "Raye Weingartner", "age": null } ] }
+{ "cid": 683, "name": "Dodie Crall", "age": 37, "address": { "number": 1337, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Cassy Crall", "age": null }, { "name": "Thu Crall", "age": 19 } ] }
+{ "cid": 684, "name": "Elmo Ballenger", "age": 69, "address": { "number": 2657, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Sheena Ballenger", "age": 53 }, { "name": "Abby Ballenger", "age": null }, { "name": "Markus Ballenger", "age": null } ] }
+{ "cid": 685, "name": "Lois Mcglothian", "age": null, "address": null, "interests": [ "Movies", "Skiing" ], "children": [ { "name": "Karon Mcglothian", "age": 35 } ] }
+{ "cid": 686, "name": "Trudi Arnette", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Adrian Arnette", "age": 43 }, { "name": "Hulda Arnette", "age": 34 }, { "name": "Shamika Arnette", "age": null } ] }
+{ "cid": 687, "name": "Adriene Glowinski", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 688, "name": "Maryellen Leriche", "age": null, "address": null, "interests": [ "Music", "Walking", "Skiing" ], "children": [ { "name": "Dorinda Leriche", "age": 27 } ] }
+{ "cid": 689, "name": "Camila Cho", "age": 70, "address": { "number": 7731, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Video Games", "Cigars" ], "children": [ { "name": "Myrtie Cho", "age": 57 }, { "name": "Merideth Cho", "age": 45 }, { "name": "Meta Cho", "age": 20 } ] }
+{ "cid": 691, "name": "Sharee Charrier", "age": 17, "address": { "number": 6693, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Bass" ], "children": [ { "name": "Odessa Charrier", "age": null } ] }
+{ "cid": 692, "name": "Nida Picknell", "age": 24, "address": { "number": 9053, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing", "Music", "Wine", "Base Jumping" ], "children": [ { "name": "Caroyln Picknell", "age": null }, { "name": "Micheline Picknell", "age": 10 } ] }
+{ "cid": 693, "name": "Ela Crisan", "age": null, "address": null, "interests": [ "Movies" ], "children": [  ] }
+{ "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] }
+{ "cid": 695, "name": "Wyatt Eveleth", "age": 28, "address": { "number": 5421, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Orval Eveleth", "age": null }, { "name": "Beth Eveleth", "age": 11 }, { "name": "Yuki Eveleth", "age": null }, { "name": "Alyse Eveleth", "age": 14 } ] }
+{ "cid": 696, "name": "Nadia Dunklee", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mendy Dunklee", "age": 17 }, { "name": "Edgar Dunklee", "age": null }, { "name": "Pasquale Dunklee", "age": null }, { "name": "Colin Dunklee", "age": null } ] }
+{ "cid": 697, "name": "Claud Coffel", "age": 72, "address": { "number": 8483, "street": "Cedar St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Katheleen Coffel", "age": 38 }, { "name": "Tashina Coffel", "age": null } ] }
+{ "cid": 698, "name": "Tawanna Zanin", "age": 60, "address": { "number": 7979, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Denny Zanin", "age": 31 }, { "name": "Danial Zanin", "age": 43 }, { "name": "Kenyetta Zanin", "age": null }, { "name": "Aleisha Zanin", "age": null } ] }
+{ "cid": 699, "name": "Lyda Golomb", "age": 46, "address": { "number": 5049, "street": "Main St.", "city": "Seattle" }, "interests": [ "Fishing", "Basketball" ], "children": [ { "name": "Shonta Golomb", "age": null }, { "name": "Lynwood Golomb", "age": 26 }, { "name": "Leonila Golomb", "age": 30 }, { "name": "Alejandrina Golomb", "age": null } ] }
+{ "cid": 700, "name": "Suk Blondin", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Brenton Blondin", "age": null }, { "name": "Charlotte Blondin", "age": null }, { "name": "Eldon Blondin", "age": 10 }, { "name": "Leanne Blondin", "age": null } ] }
+{ "cid": 702, "name": "Lane Krog", "age": 50, "address": { "number": 1646, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Carri Krog", "age": null }, { "name": "Sage Krog", "age": null }, { "name": "Bronwyn Krog", "age": null } ] }
+{ "cid": 703, "name": "Susanne Pettey", "age": null, "address": null, "interests": [ "Squash", "Basketball", "Skiing" ], "children": [ { "name": "Nancey Pettey", "age": 35 }, { "name": "Lawana Pettey", "age": null }, { "name": "Percy Pettey", "age": 25 } ] }
+{ "cid": 704, "name": "Melodee Clemons", "age": null, "address": null, "interests": [ "Base Jumping", "Tennis", "Video Games" ], "children": [ { "name": "Doreatha Clemons", "age": 22 } ] }
+{ "cid": 705, "name": "Sofia Bonniwell", "age": 81, "address": { "number": 767, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Douglass Bonniwell", "age": 58 }, { "name": "Jackeline Bonniwell", "age": 16 } ] }
+{ "cid": 706, "name": "Miquel Caesar", "age": 16, "address": { "number": 2176, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Shaniqua Caesar", "age": null }, { "name": "Ellis Caesar", "age": null }, { "name": "Bruna Caesar", "age": null }, { "name": "Kayleen Caesar", "age": null } ] }
+{ "cid": 708, "name": "Elease Holtmann", "age": 75, "address": { "number": 5295, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Leonardo Holtmann", "age": null }, { "name": "Katharine Holtmann", "age": null }, { "name": "Chung Holtmann", "age": 20 }, { "name": "Teodoro Holtmann", "age": 19 } ] }
+{ "cid": 709, "name": "Jazmine Twiddy", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Wine" ], "children": [ { "name": "Veronika Twiddy", "age": 21 } ] }
+{ "cid": 710, "name": "Arlen Horka", "age": null, "address": null, "interests": [ "Movies", "Coffee", "Walking" ], "children": [ { "name": "Valencia Horka", "age": null }, { "name": "Wesley Horka", "age": null } ] }
+{ "cid": 711, "name": "Agnes Andreas", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Fairy Andreas", "age": null }, { "name": "Wilhemina Andreas", "age": null }, { "name": "Parthenia Andreas", "age": 53 }, { "name": "Maye Andreas", "age": null } ] }
+{ "cid": 712, "name": "Jack Lamoreux", "age": 32, "address": { "number": 4486, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Rubin Lamoreux", "age": 15 }, { "name": "Jonelle Lamoreux", "age": 10 }, { "name": "Shonna Lamoreux", "age": null }, { "name": "India Lamoreux", "age": 17 } ] }
+{ "cid": 713, "name": "Galina Retterbush", "age": null, "address": null, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Janene Retterbush", "age": null }, { "name": "Toby Retterbush", "age": 15 }, { "name": "Renato Retterbush", "age": null }, { "name": "Annice Retterbush", "age": 22 } ] }
+{ "cid": 715, "name": "Zoraida Scribner", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ninfa Scribner", "age": 31 } ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 717, "name": "Paulette Moccasin", "age": 87, "address": { "number": 1426, "street": "View St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Savannah Moccasin", "age": null }, { "name": "Mariela Moccasin", "age": 34 }, { "name": "Isadora Moccasin", "age": null }, { "name": "Vivien Moccasin", "age": 31 } ] }
+{ "cid": 718, "name": "Tandy Trick", "age": 18, "address": { "number": 1215, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Fishing", "Fishing" ], "children": [ { "name": "Edyth Trick", "age": null }, { "name": "Jimmy Trick", "age": null }, { "name": "Jacquline Trick", "age": null }, { "name": "Tyler Trick", "age": null } ] }
+{ "cid": 719, "name": "Antoinette Boursiquot", "age": 47, "address": { "number": 3652, "street": "Cedar St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Dennis Boursiquot", "age": null }, { "name": "Katelyn Boursiquot", "age": null }, { "name": "Gabrielle Boursiquot", "age": null }, { "name": "Deidre Boursiquot", "age": null } ] }
+{ "cid": 721, "name": "Jesica Tinder", "age": 28, "address": { "number": 5526, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 723, "name": "Teressa Krol", "age": 22, "address": { "number": 8036, "street": "Park St.", "city": "Seattle" }, "interests": [ "Music" ], "children": [ { "name": "Tuan Krol", "age": null }, { "name": "Judi Krol", "age": null }, { "name": "Maddie Krol", "age": null } ] }
+{ "cid": 724, "name": "Merle Bakula", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Margart Bakula", "age": 49 }, { "name": "Mathew Bakula", "age": 36 } ] }
+{ "cid": 725, "name": "Sallie Calderon", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 726, "name": "Brinda Raudebaugh", "age": 83, "address": { "number": 7179, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 727, "name": "Valene Resecker", "age": null, "address": null, "interests": [ "Music", "Wine", "Books", "Walking" ], "children": [  ] }
+{ "cid": 728, "name": "Bruno Freeburger", "age": 84, "address": { "number": 2482, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Shizuko Freeburger", "age": null } ] }
+{ "cid": 730, "name": "Marti Vandoren", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Carroll Vandoren", "age": null }, { "name": "Lorretta Vandoren", "age": 30 }, { "name": "Chloe Vandoren", "age": 42 }, { "name": "Ilona Vandoren", "age": null } ] }
+{ "cid": 731, "name": "Yajaira Orto", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Eliz Orto", "age": 17 }, { "name": "Gisela Orto", "age": null } ] }
+{ "cid": 732, "name": "Dania Fabio", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Virgie Fabio", "age": null }, { "name": "Nereida Fabio", "age": 37 } ] }
+{ "cid": 733, "name": "Edie Stager", "age": 26, "address": { "number": 2691, "street": "Park St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Ethyl Stager", "age": 10 } ] }
+{ "cid": 734, "name": "Lera Korn", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Cigars" ], "children": [ { "name": "Criselda Korn", "age": 37 } ] }
+{ "cid": 736, "name": "Desmond Branam", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Manuel Branam", "age": 51 } ] }
+{ "cid": 737, "name": "Jeffrey Chesson", "age": 13, "address": { "number": 6833, "street": "Lake St.", "city": "Portland" }, "interests": [ "Tennis", "Computers" ], "children": [ { "name": "Clayton Chesson", "age": null }, { "name": "Yi Chesson", "age": null } ] }
+{ "cid": 738, "name": "Josphine Rohrer", "age": 75, "address": { "number": 862, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Marvin Rohrer", "age": 22 }, { "name": "Wyatt Rohrer", "age": null }, { "name": "Deloras Rohrer", "age": null } ] }
+{ "cid": 739, "name": "Libbie Thigpin", "age": null, "address": null, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 740, "name": "Thomasine Collado", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Tabetha Collado", "age": null }, { "name": "Alline Collado", "age": null }, { "name": "Delisa Collado", "age": null }, { "name": "Jack Collado", "age": 56 } ] }
+{ "cid": 741, "name": "Lesia Risatti", "age": 48, "address": { "number": 7378, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Fishing", "Wine", "Databases" ], "children": [ { "name": "Tangela Risatti", "age": null }, { "name": "Leonel Risatti", "age": 33 }, { "name": "Cythia Risatti", "age": 36 } ] }
+{ "cid": 742, "name": "Andy Schifo", "age": 36, "address": { "number": 4422, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 743, "name": "Nona Debroux", "age": null, "address": null, "interests": [ "Bass" ], "children": [  ] }
+{ "cid": 744, "name": "Crysta Christen", "age": 57, "address": { "number": 439, "street": "Hill St.", "city": "Portland" }, "interests": [ "Basketball", "Squash", "Base Jumping" ], "children": [  ] }
+{ "cid": 745, "name": "Tabatha Hagwell", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Gaynell Hagwell", "age": null } ] }
+{ "cid": 746, "name": "Rosalinda Pola", "age": null, "address": null, "interests": [ "Cooking", "Computers", "Walking", "Cigars" ], "children": [ { "name": "Maribel Pola", "age": 19 }, { "name": "Chaya Pola", "age": null }, { "name": "Shauna Pola", "age": null }, { "name": "Elenora Pola", "age": 22 } ] }
+{ "cid": 747, "name": "Gil Dunnaway", "age": 65, "address": { "number": 3022, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running", "Squash" ], "children": [ { "name": "Laurice Dunnaway", "age": null } ] }
+{ "cid": 748, "name": "Petra Ganes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Perry Ganes", "age": null }, { "name": "Krista Ganes", "age": 54 }, { "name": "Kayce Ganes", "age": 52 }, { "name": "Eleni Ganes", "age": null } ] }
+{ "cid": 749, "name": "Pearle Mauney", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delpha Mauney", "age": null }, { "name": "Micki Mauney", "age": 28 }, { "name": "Wayne Mauney", "age": null } ] }
+{ "cid": 750, "name": "Rosaura Gaul", "age": null, "address": null, "interests": [ "Music", "Books", "Tennis" ], "children": [ { "name": "Letisha Gaul", "age": 41 } ] }
+{ "cid": 751, "name": "Lydia Iannelli", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Teri Iannelli", "age": 36 } ] }
+{ "cid": 752, "name": "Maria Lebovic", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Thi Lebovic", "age": null }, { "name": "Rosamaria Lebovic", "age": 23 }, { "name": "Brinda Lebovic", "age": 39 } ] }
+{ "cid": 753, "name": "Maris Bannett", "age": null, "address": null, "interests": [ "Fishing", "Cigars", "Running" ], "children": [ { "name": "Libbie Bannett", "age": 11 }, { "name": "Francina Bannett", "age": 21 }, { "name": "Tuyet Bannett", "age": null }, { "name": "Zona Bannett", "age": 32 } ] }
+{ "cid": 754, "name": "Luetta Joern", "age": 25, "address": { "number": 5554, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Hildegarde Joern", "age": null }, { "name": "Lorenza Joern", "age": 13 } ] }
+{ "cid": 755, "name": "Bette Trentz", "age": 57, "address": { "number": 2794, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Christa Trentz", "age": 14 }, { "name": "Jestine Trentz", "age": 22 }, { "name": "Shantel Trentz", "age": 37 }, { "name": "Jacklyn Trentz", "age": null } ] }
+{ "cid": 756, "name": "Marisol Noyes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delora Noyes", "age": null }, { "name": "Jonelle Noyes", "age": 44 } ] }
+{ "cid": 758, "name": "Akiko Hoenstine", "age": 56, "address": { "number": 8888, "street": "Lake St.", "city": "Portland" }, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Maren Hoenstine", "age": null }, { "name": "Tyler Hoenstine", "age": null }, { "name": "Jesse Hoenstine", "age": 40 } ] }
+{ "cid": 759, "name": "Alaina Dadds", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Athena Dadds", "age": 36 }, { "name": "Denis Dadds", "age": null }, { "name": "Nathanial Dadds", "age": 42 }, { "name": "Molly Dadds", "age": null } ] }
+{ "cid": 761, "name": "Adele Henrikson", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Paulina Henrikson", "age": null }, { "name": "David Henrikson", "age": null }, { "name": "Jose Henrikson", "age": null }, { "name": "Meg Henrikson", "age": null } ] }
+{ "cid": 763, "name": "Candis Deya", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lise Deya", "age": null }, { "name": "Jeni Deya", "age": 52 }, { "name": "Domonique Deya", "age": 24 }, { "name": "Rubie Deya", "age": null } ] }
+{ "cid": 766, "name": "Tosha Loffredo", "age": 64, "address": { "number": 5580, "street": "View St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Hellen Loffredo", "age": 32 } ] }
+{ "cid": 767, "name": "Wendi Hoecker", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 768, "name": "Adelina Troendle", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lenna Troendle", "age": 51 }, { "name": "Ines Troendle", "age": 48 }, { "name": "Ora Troendle", "age": null } ] }
+{ "cid": 769, "name": "Isaias Tenny", "age": 71, "address": { "number": 270, "street": "Park St.", "city": "Portland" }, "interests": [ "Wine", "Fishing", "Base Jumping" ], "children": [ { "name": "Theo Tenny", "age": null }, { "name": "Shena Tenny", "age": null }, { "name": "Coralee Tenny", "age": null }, { "name": "Orval Tenny", "age": 39 } ] }
+{ "cid": 770, "name": "Merrill Tilson", "age": null, "address": null, "interests": [ "Computers", "Skiing" ], "children": [ { "name": "Elna Tilson", "age": null } ] }
+{ "cid": 771, "name": "Marisela Tredo", "age": null, "address": null, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Ardell Tredo", "age": 21 }, { "name": "Evelynn Tredo", "age": 16 } ] }
+{ "cid": 773, "name": "Leatrice Zysett", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Bee Zysett", "age": 30 }, { "name": "Russ Zysett", "age": 11 }, { "name": "Jeff Zysett", "age": 39 }, { "name": "Herman Zysett", "age": 27 } ] }
+{ "cid": 774, "name": "Nadene Rigel", "age": null, "address": null, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Rebbeca Rigel", "age": 33 } ] }
+{ "cid": 776, "name": "Dagmar Sarkis", "age": null, "address": null, "interests": [ "Basketball", "Running", "Wine" ], "children": [ { "name": "Tari Sarkis", "age": null }, { "name": "Rana Sarkis", "age": 56 }, { "name": "Merissa Sarkis", "age": null }, { "name": "Lori Sarkis", "age": 26 } ] }
+{ "cid": 777, "name": "Coralee Vaugh", "age": 51, "address": { "number": 4130, "street": "Hill St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Dean Vaugh", "age": 31 }, { "name": "Stanton Vaugh", "age": 39 }, { "name": "Marti Vaugh", "age": 33 }, { "name": "Eden Vaugh", "age": 27 } ] }
+{ "cid": 778, "name": "Shellie Sario", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 779, "name": "Vinita Bockskopf", "age": null, "address": null, "interests": [ "Tennis", "Video Games" ], "children": [  ] }
+{ "cid": 780, "name": "Penny Poortinga", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Estella Poortinga", "age": null } ] }
+{ "cid": 781, "name": "Christy Darcangelo", "age": 42, "address": { "number": 2178, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Luis Darcangelo", "age": 21 }, { "name": "Omega Darcangelo", "age": 26 }, { "name": "Remedios Darcangelo", "age": 28 }, { "name": "Domenic Darcangelo", "age": 21 } ] }
+{ "cid": 782, "name": "Shameka Haifa", "age": 16, "address": { "number": 9555, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Cigars", "Computers", "Coffee", "Skiing" ], "children": [ { "name": "Dannette Haifa", "age": null } ] }
+{ "cid": 783, "name": "Johnnie Kesby", "age": 56, "address": { "number": 9798, "street": "View St.", "city": "Seattle" }, "interests": [ "Puzzles", "Tennis" ], "children": [  ] }
+{ "cid": 784, "name": "Omar Hasen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Hugh Hasen", "age": null } ] }
+{ "cid": 785, "name": "Gabriel Breidel", "age": 32, "address": { "number": 9288, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cigars", "Bass" ], "children": [ { "name": "Bernie Breidel", "age": null } ] }
+{ "cid": 786, "name": "Johnsie Maheux", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Danuta Maheux", "age": null } ] }
+{ "cid": 787, "name": "Sara Yerly", "age": 12, "address": { "number": 872, "street": "7th St.", "city": "Seattle" }, "interests": [ "Fishing" ], "children": [ { "name": "Nettie Yerly", "age": null }, { "name": "Regine Yerly", "age": null }, { "name": "Hyo Yerly", "age": null } ] }
+{ "cid": 789, "name": "Carli Notto", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 790, "name": "Dustin Brumble", "age": null, "address": null, "interests": [ "Computers", "Databases", "Tennis" ], "children": [ { "name": "Oda Brumble", "age": null }, { "name": "Jennefer Brumble", "age": 26 }, { "name": "Ricardo Brumble", "age": 37 }, { "name": "Graciela Brumble", "age": 10 } ] }
+{ "cid": 791, "name": "Jame Apresa", "age": 66, "address": { "number": 8417, "street": "Main St.", "city": "San Jose" }, "interests": [ "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Awilda Apresa", "age": null }, { "name": "Nelle Apresa", "age": 40 }, { "name": "Terrell Apresa", "age": null }, { "name": "Malia Apresa", "age": 43 } ] }
+{ "cid": 793, "name": "Shondra Gollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Paul Gollman", "age": 30 }, { "name": "Katherina Gollman", "age": 53 } ] }
+{ "cid": 794, "name": "Annabel Leins", "age": 75, "address": { "number": 9761, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Bass", "Computers", "Bass", "Cigars" ], "children": [ { "name": "Oswaldo Leins", "age": 21 } ] }
+{ "cid": 795, "name": "Sharilyn Branstad", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ashlee Branstad", "age": 24 }, { "name": "Bobbye Branstad", "age": 26 }, { "name": "Natalya Branstad", "age": null }, { "name": "Edith Branstad", "age": null } ] }
+{ "cid": 796, "name": "Daniele Brisk", "age": null, "address": null, "interests": [ "Walking", "Bass" ], "children": [  ] }
+{ "cid": 797, "name": "Frederica Kale", "age": 77, "address": { "number": 6861, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Bass" ], "children": [ { "name": "Shanice Kale", "age": null }, { "name": "Soraya Kale", "age": 64 }, { "name": "Laurena Kale", "age": 57 } ] }
+{ "cid": 799, "name": "Ronny Piefer", "age": 45, "address": { "number": 7724, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Chantal Piefer", "age": 24 }, { "name": "Tiffany Piefer", "age": null }, { "name": "Farrah Piefer", "age": 21 }, { "name": "Dee Piefer", "age": null } ] }
+{ "cid": 800, "name": "Karon Johnsen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Roselee Johnsen", "age": 25 } ] }
+{ "cid": 802, "name": "Sang Hollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Carman Hollman", "age": null }, { "name": "Kirstie Hollman", "age": 40 }, { "name": "Jacquetta Hollman", "age": null } ] }
+{ "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] }
+{ "cid": 804, "name": "Joaquina Burlin", "age": 77, "address": { "number": 5479, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Running", "Wine", "Running" ], "children": [  ] }
+{ "cid": 805, "name": "Gaylord Ginder", "age": null, "address": null, "interests": [ "Databases", "Coffee" ], "children": [ { "name": "Lucina Ginder", "age": null }, { "name": "Harriett Ginder", "age": null } ] }
+{ "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] }
+{ "cid": 807, "name": "Maryanne Kuzminski", "age": 21, "address": { "number": 1601, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Running" ], "children": [ { "name": "India Kuzminski", "age": null }, { "name": "Adell Kuzminski", "age": null } ] }
+{ "cid": 808, "name": "Brande Decius", "age": null, "address": null, "interests": [ "Basketball", "Fishing", "Puzzles" ], "children": [ { "name": "Li Decius", "age": 56 }, { "name": "Eusebio Decius", "age": 50 }, { "name": "Clementina Decius", "age": 29 } ] }
+{ "cid": 809, "name": "Dagny Mangiaracina", "age": 44, "address": { "number": 5993, "street": "Lake St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Bari Mangiaracina", "age": 31 }, { "name": "Tiara Mangiaracina", "age": 12 }, { "name": "Milly Mangiaracina", "age": null }, { "name": "Chelsie Mangiaracina", "age": null } ] }
+{ "cid": 810, "name": "Myron Dumlao", "age": null, "address": null, "interests": [ "Wine", "Coffee" ], "children": [ { "name": "Josie Dumlao", "age": 36 } ] }
+{ "cid": 811, "name": "Marti Whitmyre", "age": null, "address": null, "interests": [ "Music", "Walking" ], "children": [  ] }
+{ "cid": 812, "name": "Bee Godette", "age": 26, "address": { "number": 1757, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Base Jumping", "Tennis" ], "children": [ { "name": "Madaline Godette", "age": 10 }, { "name": "Shasta Godette", "age": 15 }, { "name": "Parthenia Godette", "age": 11 }, { "name": "Priscila Godette", "age": 13 } ] }
+{ "cid": 813, "name": "Leann Domagala", "age": 47, "address": { "number": 4472, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Alvera Domagala", "age": 36 }, { "name": "Rosalva Domagala", "age": 27 }, { "name": "Eugenia Domagala", "age": null }, { "name": "My Domagala", "age": 32 } ] }
+{ "cid": 814, "name": "Harriette Kasmarek", "age": 68, "address": { "number": 7191, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Skiing" ], "children": [ { "name": "Melani Kasmarek", "age": 24 }, { "name": "Jesica Kasmarek", "age": 22 } ] }
+{ "cid": 815, "name": "Leigha Bires", "age": 11, "address": { "number": 7263, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running" ], "children": [ { "name": "Val Bires", "age": null } ] }
+{ "cid": 816, "name": "Cheyenne Eddie", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Kathe Eddie", "age": null }, { "name": "Charles Eddie", "age": null } ] }
+{ "cid": 818, "name": "Nellie Whetzell", "age": null, "address": null, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 819, "name": "Twanna Finnley", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [ { "name": "Reba Finnley", "age": null }, { "name": "Moises Finnley", "age": null } ] }
+{ "cid": 820, "name": "Lacy Caudill", "age": 22, "address": { "number": 8679, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Sybil Caudill", "age": null } ] }
+{ "cid": 821, "name": "Carole Edlund", "age": 76, "address": { "number": 4008, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Computers", "Cooking", "Running", "Basketball" ], "children": [ { "name": "Garfield Edlund", "age": 54 }, { "name": "Brooks Edlund", "age": null }, { "name": "Gertrudis Edlund", "age": null }, { "name": "Tabitha Edlund", "age": 58 } ] }
+{ "cid": 824, "name": "Vonda Czaplewski", "age": 72, "address": { "number": 4597, "street": "7th St.", "city": "Portland" }, "interests": [ "Skiing" ], "children": [ { "name": "Gaynelle Czaplewski", "age": null }, { "name": "India Czaplewski", "age": null } ] }
+{ "cid": 825, "name": "Kirstie Rinebold", "age": 57, "address": { "number": 9463, "street": "Oak St.", "city": "Portland" }, "interests": [ "Cooking", "Cigars", "Books" ], "children": [ { "name": "Vonda Rinebold", "age": null }, { "name": "Man Rinebold", "age": 21 } ] }
+{ "cid": 826, "name": "Ressie Feenstra", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Sasha Feenstra", "age": null } ] }
+{ "cid": 827, "name": "Clementina Papin", "age": null, "address": null, "interests": [ "Music", "Basketball", "Cigars" ], "children": [ { "name": "Catina Papin", "age": null }, { "name": "Demetrius Papin", "age": 59 }, { "name": "Marylou Papin", "age": 12 }, { "name": "Apryl Papin", "age": 16 } ] }
+{ "cid": 828, "name": "Marcelle Steinhour", "age": null, "address": null, "interests": [ "Running", "Basketball", "Walking" ], "children": [ { "name": "Jimmie Steinhour", "age": 13 }, { "name": "Kirstie Steinhour", "age": 19 } ] }
+{ "cid": 831, "name": "Raina Rys", "age": 62, "address": { "number": 7048, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Walking" ], "children": [ { "name": "Ezra Rys", "age": null }, { "name": "Carl Rys", "age": null }, { "name": "Loraine Rys", "age": null } ] }
+{ "cid": 832, "name": "Alina Hosley", "age": null, "address": null, "interests": [ "Databases", "Databases", "Music" ], "children": [ { "name": "Sebrina Hosley", "age": null }, { "name": "Dyan Hosley", "age": null } ] }
+{ "cid": 833, "name": "Lakisha Petkoff", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittanie Petkoff", "age": null }, { "name": "Ashli Petkoff", "age": null } ] }
+{ "cid": 834, "name": "Luvenia Grandstaff", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Joleen Grandstaff", "age": 28 }, { "name": "Elvera Grandstaff", "age": null }, { "name": "Leonia Grandstaff", "age": 35 }, { "name": "Jaclyn Grandstaff", "age": 28 } ] }
+{ "cid": 835, "name": "Raphael Marzili", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Angelic Marzili", "age": 38 } ] }
+{ "cid": 836, "name": "Elden Shumski", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Weldon Shumski", "age": null }, { "name": "Anneliese Shumski", "age": null } ] }
+{ "cid": 837, "name": "Denice Wolken", "age": 28, "address": { "number": 5010, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Kattie Wolken", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
+{ "cid": 841, "name": "Omar Enwall", "age": null, "address": null, "interests": [ "Skiing", "Skiing", "Books" ], "children": [ { "name": "Kirby Enwall", "age": 31 }, { "name": "Cythia Enwall", "age": 24 }, { "name": "August Enwall", "age": null } ] }
+{ "cid": 843, "name": "Lenny Acerno", "age": 64, "address": { "number": 7656, "street": "Main St.", "city": "Seattle" }, "interests": [ "Base Jumping", "Squash" ], "children": [  ] }
+{ "cid": 844, "name": "Madelene Ten", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Johanne Ten", "age": 39 }, { "name": "Lurline Ten", "age": null }, { "name": "Cathy Ten", "age": 49 } ] }
+{ "cid": 845, "name": "Burt Earp", "age": 21, "address": { "number": 7626, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Computers" ], "children": [ { "name": "Denny Earp", "age": null }, { "name": "Blaine Earp", "age": null }, { "name": "Wilson Earp", "age": 10 }, { "name": "Joan Earp", "age": null } ] }
+{ "cid": 846, "name": "Kieth Norlund", "age": 15, "address": { "number": 4039, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Puzzles" ], "children": [ { "name": "Shawn Norlund", "age": null } ] }
+{ "cid": 847, "name": "Ashton Korba", "age": 25, "address": { "number": 6450, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Computers", "Walking", "Video Games" ], "children": [  ] }
+{ "cid": 848, "name": "Myrta Kopf", "age": null, "address": null, "interests": [ "Wine", "Basketball", "Base Jumping" ], "children": [  ] }
+{ "cid": 850, "name": "Garnet Younce", "age": null, "address": null, "interests": [ "Databases", "Video Games", "Books" ], "children": [ { "name": "Syble Younce", "age": 16 } ] }
+{ "cid": 851, "name": "Darrel Machia", "age": 31, "address": { "number": 3290, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Coy Machia", "age": 13 }, { "name": "Janean Machia", "age": 13 }, { "name": "Sandi Machia", "age": 18 } ] }
+{ "cid": 852, "name": "Terrell Ramsay", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 853, "name": "Denisse Peralto", "age": 25, "address": { "number": 3931, "street": "7th St.", "city": "Portland" }, "interests": [ "Tennis", "Walking", "Basketball" ], "children": [ { "name": "Asha Peralto", "age": 14 }, { "name": "Clark Peralto", "age": null }, { "name": "Jessika Peralto", "age": null }, { "name": "Nadene Peralto", "age": null } ] }
+{ "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] }
+{ "cid": 855, "name": "Rosette Reen", "age": 57, "address": { "number": 2767, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 857, "name": "Kasie Fujioka", "age": null, "address": null, "interests": [ "Skiing", "Cigars" ], "children": [ { "name": "Leontine Fujioka", "age": null }, { "name": "Nga Fujioka", "age": 21 }, { "name": "Nathanael Fujioka", "age": 27 } ] }
+{ "cid": 858, "name": "Maricruz Dittberner", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Cigars", "Video Games" ], "children": [  ] }
+{ "cid": 859, "name": "Mozelle Catillo", "age": 61, "address": { "number": 253, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Databases", "Cooking", "Wine" ], "children": [  ] }
+{ "cid": 860, "name": "Isabelle Sept", "age": 88, "address": { "number": 4382, "street": "Washington St.", "city": "Portland" }, "interests": [ "Puzzles", "Books" ], "children": [  ] }
+{ "cid": 861, "name": "Hugh Mcbrien", "age": null, "address": null, "interests": [ "Skiing", "Cigars", "Cooking" ], "children": [ { "name": "Otha Mcbrien", "age": 38 } ] }
+{ "cid": 862, "name": "Constance Bries", "age": 77, "address": { "number": 2585, "street": "Oak St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Lizzie Bries", "age": 42 }, { "name": "Shenika Bries", "age": null }, { "name": "Phillip Bries", "age": null } ] }
+{ "cid": 864, "name": "Katharyn Zanotti", "age": 62, "address": { "number": 8336, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Magan Zanotti", "age": null }, { "name": "Jacinto Zanotti", "age": null } ] }
+{ "cid": 865, "name": "Moon Marino", "age": 43, "address": { "number": 5710, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Markita Marino", "age": 10 } ] }
+{ "cid": 866, "name": "Bonita Kauphusman", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 869, "name": "Lino Wooderson", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nola Wooderson", "age": null }, { "name": "Leticia Wooderson", "age": 36 }, { "name": "Bernardine Wooderson", "age": null } ] }
+{ "cid": 870, "name": "Natosha Lufsey", "age": null, "address": null, "interests": [ "Cigars", "Walking" ], "children": [ { "name": "Tiffany Lufsey", "age": null } ] }
+{ "cid": 871, "name": "Lona Dacus", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Pablo Dacus", "age": null }, { "name": "Darlene Dacus", "age": 45 }, { "name": "Darius Dacus", "age": 31 }, { "name": "Cordia Dacus", "age": null } ] }
+{ "cid": 872, "name": "Michele Herschel", "age": 39, "address": { "number": 4287, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] }
+{ "cid": 876, "name": "Chelsie Motten", "age": null, "address": null, "interests": [ "Music", "Squash", "Music", "Walking" ], "children": [ { "name": "Nida Motten", "age": null }, { "name": "Taneka Motten", "age": 10 }, { "name": "Maynard Motten", "age": 57 } ] }
+{ "cid": 877, "name": "Nicki Lipkind", "age": null, "address": null, "interests": [ "Books", "Movies" ], "children": [ { "name": "Yahaira Lipkind", "age": 12 } ] }
+{ "cid": 878, "name": "Migdalia Bisker", "age": 50, "address": { "number": 6699, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Computers", "Basketball" ], "children": [ { "name": "Moira Bisker", "age": null }, { "name": "Tanisha Bisker", "age": null } ] }
+{ "cid": 879, "name": "Vinnie Antoniewicz", "age": 45, "address": { "number": 1633, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Cooking", "Puzzles" ], "children": [  ] }
+{ "cid": 880, "name": "Sara Abo", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 881, "name": "Leora Chesnutt", "age": 49, "address": { "number": 6487, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Movies" ], "children": [ { "name": "Myrtle Chesnutt", "age": null }, { "name": "Serina Chesnutt", "age": 11 }, { "name": "Jana Chesnutt", "age": 10 } ] }
+{ "cid": 883, "name": "Odilia Bugtong", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mark Bugtong", "age": 15 }, { "name": "Paula Bugtong", "age": null }, { "name": "Jenee Bugtong", "age": 17 }, { "name": "Lilian Bugtong", "age": 44 } ] }
+{ "cid": 884, "name": "Laila Marta", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [ { "name": "Carlota Marta", "age": 19 } ] }
+{ "cid": 885, "name": "Les Legere", "age": 87, "address": { "number": 3998, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Bass", "Tennis", "Fishing" ], "children": [ { "name": "Concetta Legere", "age": 45 }, { "name": "Tamica Legere", "age": null }, { "name": "Aurora Legere", "age": null } ] }
+{ "cid": 887, "name": "Jermaine Folz", "age": 35, "address": { "number": 8487, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Computers", "Puzzles", "Cooking" ], "children": [ { "name": "Sharice Folz", "age": null } ] }
+{ "cid": 888, "name": "Natalie Nocella", "age": 66, "address": { "number": 2856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Noel Nocella", "age": 26 }, { "name": "Damon Nocella", "age": 29 }, { "name": "Joesph Nocella", "age": 33 }, { "name": "Nidia Nocella", "age": null } ] }
+{ "cid": 889, "name": "Elvis Schoff", "age": 83, "address": { "number": 6724, "street": "Hill St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Spring Schoff", "age": 43 }, { "name": "Davis Schoff", "age": 55 }, { "name": "Ryann Schoff", "age": 58 }, { "name": "Clarinda Schoff", "age": 11 } ] }
+{ "cid": 890, "name": "Janise Maccarthy", "age": 66, "address": { "number": 7337, "street": "Main St.", "city": "San Jose" }, "interests": [ "Wine", "Computers" ], "children": [  ] }
+{ "cid": 891, "name": "Jesusita Bhatia", "age": 57, "address": { "number": 1476, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 892, "name": "Madge Hendson", "age": 79, "address": { "number": 8832, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Fishing", "Skiing" ], "children": [ { "name": "Elia Hendson", "age": 48 }, { "name": "Lashawn Hendson", "age": 27 } ] }
+{ "cid": 893, "name": "Norberto Banchero", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 894, "name": "Reginald Julien", "age": 16, "address": { "number": 1107, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases", "Wine" ], "children": [ { "name": "Arthur Julien", "age": null }, { "name": "Evia Julien", "age": null } ] }
+{ "cid": 897, "name": "Gerald Roehrman", "age": null, "address": null, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Virgie Roehrman", "age": 28 }, { "name": "Akiko Roehrman", "age": 59 }, { "name": "Robbie Roehrman", "age": 10 }, { "name": "Flavia Roehrman", "age": null } ] }
+{ "cid": 898, "name": "Thao Seufert", "age": 78, "address": { "number": 3529, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Bass", "Squash", "Coffee" ], "children": [ { "name": "Classie Seufert", "age": null } ] }
+{ "cid": 899, "name": "Ada Kamealoha", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Juliann Kamealoha", "age": null }, { "name": "Ilana Kamealoha", "age": 25 }, { "name": "Herminia Kamealoha", "age": 55 }, { "name": "Carli Kamealoha", "age": null } ] }
+{ "cid": 901, "name": "Riva Ziko", "age": null, "address": null, "interests": [ "Running", "Tennis", "Video Games" ], "children": [ { "name": "Leandra Ziko", "age": 49 }, { "name": "Torrie Ziko", "age": null } ] }
+{ "cid": 903, "name": "Elise Morenz", "age": 17, "address": { "number": 8968, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 904, "name": "Holley Tofil", "age": 51, "address": { "number": 8946, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Kristal Tofil", "age": null } ] }
+{ "cid": 905, "name": "Pandora Azzarella", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lane Azzarella", "age": null }, { "name": "Joi Azzarella", "age": 19 } ] }
+{ "cid": 907, "name": "Princess Sudol", "age": 73, "address": { "number": 9770, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Base Jumping" ], "children": [ { "name": "Bronwyn Sudol", "age": 22 }, { "name": "Judith Sudol", "age": null } ] }
+{ "cid": 908, "name": "Ferdinand Auila", "age": 82, "address": { "number": 1071, "street": "Lake St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running", "Wine" ], "children": [ { "name": "Ai Auila", "age": 69 }, { "name": "Laurel Auila", "age": null } ] }
+{ "cid": 909, "name": "Mariko Sharar", "age": null, "address": null, "interests": [ "Squash", "Movies", "Computers" ], "children": [  ] }
+{ "cid": 910, "name": "Everette Moe", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Berna Moe", "age": 56 }, { "name": "Harold Moe", "age": 28 }, { "name": "See Moe", "age": 20 } ] }
+{ "cid": 911, "name": "Eileen Bartolomeo", "age": 20, "address": { "number": 8915, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 912, "name": "Alessandra Kaskey", "age": 52, "address": { "number": 6906, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Skiing", "Walking", "Basketball" ], "children": [ { "name": "Mack Kaskey", "age": null } ] }
+{ "cid": 913, "name": "Evelynn Fague", "age": 42, "address": { "number": 5729, "street": "7th St.", "city": "Seattle" }, "interests": [ "Books", "Databases", "Cooking" ], "children": [  ] }
+{ "cid": 914, "name": "Hunter Flournoy", "age": null, "address": null, "interests": [ "Cooking", "Squash" ], "children": [ { "name": "Christopher Flournoy", "age": 59 }, { "name": "Earnestine Flournoy", "age": null } ] }
+{ "cid": 916, "name": "Kris Mcmarlin", "age": null, "address": null, "interests": [ "Movies", "Music", "Puzzles" ], "children": [  ] }
+{ "cid": 917, "name": "Jerri Blachowski", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Chet Blachowski", "age": 43 }, { "name": "Mallory Blachowski", "age": null }, { "name": "Akilah Blachowski", "age": null } ] }
+{ "cid": 919, "name": "Fairy Wansley", "age": 45, "address": { "number": 9020, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Databases", "Video Games" ], "children": [ { "name": "Marvella Wansley", "age": null }, { "name": "Hisako Wansley", "age": null }, { "name": "Shaunta Wansley", "age": null }, { "name": "Gemma Wansley", "age": 21 } ] }
+{ "cid": 920, "name": "Mirtha Dellbringge", "age": null, "address": null, "interests": [ "Walking", "Basketball", "Basketball" ], "children": [ { "name": "Morgan Dellbringge", "age": 51 }, { "name": "Alease Dellbringge", "age": 35 } ] }
+{ "cid": 921, "name": "Mario Nolden", "age": 17, "address": { "number": 3977, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Gertrude Nolden", "age": null }, { "name": "Ray Nolden", "age": null }, { "name": "Inocencia Nolden", "age": null } ] }
+{ "cid": 922, "name": "Shanice Lingle", "age": 26, "address": { "number": 4753, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Sandie Lingle", "age": 12 }, { "name": "Nia Lingle", "age": 13 }, { "name": "Marilyn Lingle", "age": 15 } ] }
+{ "cid": 923, "name": "Bobbi Ursino", "age": null, "address": null, "interests": [ "Movies", "Books", "Walking" ], "children": [ { "name": "Shon Ursino", "age": null }, { "name": "Lorean Ursino", "age": null } ] }
+{ "cid": 924, "name": "Kathleen Lash", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Clementina Lash", "age": 58 }, { "name": "Zula Lash", "age": null }, { "name": "Mellissa Lash", "age": 54 } ] }
+{ "cid": 925, "name": "Quintin Kizzie", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Bass", "Movies" ], "children": [ { "name": "Julius Kizzie", "age": 11 }, { "name": "Melissia Kizzie", "age": null }, { "name": "Olga Kizzie", "age": 42 } ] }
+{ "cid": 927, "name": "Lillia Hartlein", "age": 55, "address": { "number": 5856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Coffee", "Cigars" ], "children": [ { "name": "Nicky Hartlein", "age": null }, { "name": "Cassaundra Hartlein", "age": 10 }, { "name": "Micheline Hartlein", "age": 26 }, { "name": "Anton Hartlein", "age": 32 } ] }
+{ "cid": 928, "name": "Maddie Diclaudio", "age": 33, "address": { "number": 4674, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Databases", "Bass" ], "children": [ { "name": "Dominique Diclaudio", "age": 12 } ] }
+{ "cid": 929, "name": "Jean Guitierrez", "age": 75, "address": { "number": 9736, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Wine", "Wine", "Fishing" ], "children": [  ] }
+{ "cid": 930, "name": "Kathie Gier", "age": 37, "address": { "number": 5075, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Onie Gier", "age": 16 } ] }
+{ "cid": 931, "name": "Octavia Koiner", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ardath Koiner", "age": 32 }, { "name": "Milly Koiner", "age": null }, { "name": "Arlinda Koiner", "age": null }, { "name": "Debby Koiner", "age": null } ] }
+{ "cid": 932, "name": "Kraig Bomia", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 933, "name": "Eartha Hershberger", "age": 81, "address": { "number": 7013, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles" ], "children": [ { "name": "Waneta Hershberger", "age": null }, { "name": "Katherine Hershberger", "age": 67 }, { "name": "Johnnie Hershberger", "age": 25 }, { "name": "Jovan Hershberger", "age": 30 } ] }
+{ "cid": 934, "name": "Dessie Lockmiller", "age": 70, "address": { "number": 4313, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Coffee", "Puzzles" ], "children": [  ] }
+{ "cid": 935, "name": "Sharita Aspegren", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Russell Aspegren", "age": 35 }, { "name": "Bernardina Aspegren", "age": null }, { "name": "Isobel Aspegren", "age": 11 }, { "name": "Reva Aspegren", "age": null } ] }
+{ "cid": 937, "name": "Annika Pauline", "age": 78, "address": { "number": 8563, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Mikki Pauline", "age": 34 } ] }
+{ "cid": 938, "name": "Parthenia Dromgoole", "age": 36, "address": { "number": 527, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [  ] }
+{ "cid": 940, "name": "Kitty Nalepka", "age": null, "address": null, "interests": [ "Movies", "Wine", "Basketball" ], "children": [ { "name": "Kendra Nalepka", "age": null } ] }
+{ "cid": 941, "name": "Jamey Jakobson", "age": null, "address": null, "interests": [ "Books", "Cooking", "Video Games" ], "children": [ { "name": "Elmer Jakobson", "age": 14 }, { "name": "Minh Jakobson", "age": 30 } ] }
+{ "cid": 942, "name": "Emerson Keblish", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Leonora Keblish", "age": null } ] }
+{ "cid": 943, "name": "Kathryne Blacock", "age": 82, "address": { "number": 3510, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Bass", "Music" ], "children": [  ] }
+{ "cid": 944, "name": "Johana Hisman", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Kirstin Hisman", "age": 43 }, { "name": "Darwin Hisman", "age": 29 } ] }
+{ "cid": 945, "name": "Hildegard Dedinas", "age": 70, "address": { "number": 3273, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Renato Dedinas", "age": 35 } ] }
+{ "cid": 946, "name": "Taylor Parrigan", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Salome Parrigan", "age": 50 }, { "name": "Gary Parrigan", "age": 25 }, { "name": "Harold Parrigan", "age": null } ] }
+{ "cid": 948, "name": "Thad Scialpi", "age": 22, "address": { "number": 8731, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Tennis", "Wine" ], "children": [ { "name": "Harlan Scialpi", "age": 10 }, { "name": "Lucile Scialpi", "age": 11 }, { "name": "Audria Scialpi", "age": null } ] }
+{ "cid": 949, "name": "Elissa Rogue", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Noriko Rogue", "age": 41 }, { "name": "Lavona Rogue", "age": 39 } ] }
+{ "cid": 950, "name": "Young Bayn", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Evangeline Bayn", "age": 38 }, { "name": "Darcy Bayn", "age": 45 }, { "name": "Rosita Bayn", "age": null }, { "name": "Austin Bayn", "age": 46 } ] }
+{ "cid": 951, "name": "Janine Martorano", "age": 65, "address": { "number": 6420, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Books", "Music" ], "children": [ { "name": "Idella Martorano", "age": null } ] }
+{ "cid": 955, "name": "Liliana Stenkamp", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 956, "name": "Laquanda Bynoe", "age": 79, "address": { "number": 6122, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Joel Bynoe", "age": null }, { "name": "Brian Bynoe", "age": 61 }, { "name": "Shana Bynoe", "age": null } ] }
+{ "cid": 957, "name": "Lucius Schurr", "age": 75, "address": { "number": 3918, "street": "Main St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Willetta Schurr", "age": 22 }, { "name": "Andre Schurr", "age": null }, { "name": "Merrilee Schurr", "age": 32 } ] }
+{ "cid": 958, "name": "Ricardo Pezzica", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delois Pezzica", "age": 11 } ] }
+{ "cid": 960, "name": "Lenore Limardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Kris Limardi", "age": 12 } ] }
+{ "cid": 961, "name": "Mirian Herpolsheimer", "age": null, "address": null, "interests": [ "Music", "Fishing", "Computers" ], "children": [ { "name": "Larissa Herpolsheimer", "age": 41 }, { "name": "Markus Herpolsheimer", "age": null }, { "name": "Natacha Herpolsheimer", "age": null } ] }
+{ "cid": 962, "name": "Taryn Coley", "age": null, "address": null, "interests": [ "Running", "Basketball", "Cooking" ], "children": [  ] }
+{ "cid": 963, "name": "Mila Ditmars", "age": 29, "address": { "number": 5850, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 964, "name": "Stephany Soders", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Computers" ], "children": [  ] }
+{ "cid": 965, "name": "Mellie Risen", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Coreen Risen", "age": 36 }, { "name": "Faith Risen", "age": 34 }, { "name": "Crystle Risen", "age": 54 } ] }
+{ "cid": 966, "name": "Brigitte Quimby", "age": 13, "address": { "number": 203, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Skiing", "Tennis" ], "children": [ { "name": "Ilona Quimby", "age": null }, { "name": "Shaunte Quimby", "age": null }, { "name": "Lorie Quimby", "age": null } ] }
+{ "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] }
+{ "cid": 970, "name": "Pia Sudderth", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Ernestina Sudderth", "age": 15 }, { "name": "Larue Sudderth", "age": 46 }, { "name": "Toshia Sudderth", "age": 27 } ] }
+{ "cid": 974, "name": "Alexis Malcomson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Kerri Malcomson", "age": null } ] }
+{ "cid": 975, "name": "Gary Whitemore", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 976, "name": "Madalyn Nidiffer", "age": 35, "address": { "number": 7635, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Wine", "Music" ], "children": [ { "name": "Tricia Nidiffer", "age": 10 }, { "name": "Kevin Nidiffer", "age": 24 }, { "name": "Elyse Nidiffer", "age": null } ] }
+{ "cid": 978, "name": "Rudy Watsky", "age": 32, "address": { "number": 2754, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Cooking" ], "children": [  ] }
+{ "cid": 979, "name": "Yoko Bailony", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Vivienne Bailony", "age": null }, { "name": "Lori Bailony", "age": 47 } ] }
+{ "cid": 980, "name": "Harley Lappe", "age": 56, "address": { "number": 647, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Books", "Cigars", "Basketball" ], "children": [ { "name": "Maxwell Lappe", "age": null }, { "name": "Gemma Lappe", "age": 32 }, { "name": "Ester Lappe", "age": 40 }, { "name": "Myles Lappe", "age": 36 } ] }
+{ "cid": 981, "name": "Lilliam Lopus", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Tracey Lopus", "age": null } ] }
+{ "cid": 982, "name": "Jude Brandsrud", "age": 41, "address": { "number": 7133, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Skiing" ], "children": [ { "name": "Scottie Brandsrud", "age": null }, { "name": "Gennie Brandsrud", "age": 10 }, { "name": "Agnes Brandsrud", "age": null }, { "name": "Clarinda Brandsrud", "age": 17 } ] }
+{ "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] }
+{ "cid": 985, "name": "Arnette Farlow", "age": 23, "address": { "number": 7843, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Databases" ], "children": [ { "name": "Lora Farlow", "age": 12 }, { "name": "Arlen Farlow", "age": 11 }, { "name": "Rodney Farlow", "age": null }, { "name": "Tori Farlow", "age": 11 } ] }
+{ "cid": 986, "name": "Tennille Wikle", "age": 78, "address": { "number": 3428, "street": "View St.", "city": "Portland" }, "interests": [ "Movies", "Databases", "Wine" ], "children": [ { "name": "Lourie Wikle", "age": null }, { "name": "Laure Wikle", "age": null } ] }
+{ "cid": 987, "name": "Sharolyn Demchak", "age": 36, "address": { "number": 4672, "street": "Lake St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 988, "name": "Dagmar Plasky", "age": 89, "address": { "number": 1219, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Dann Plasky", "age": 59 }, { "name": "Raye Plasky", "age": null }, { "name": "Sammie Plasky", "age": 36 }, { "name": "Kasi Plasky", "age": 24 } ] }
+{ "cid": 991, "name": "Leonel Toepperwein", "age": 62, "address": { "number": 8356, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Coffee", "Books" ], "children": [ { "name": "Sean Toepperwein", "age": null }, { "name": "Charline Toepperwein", "age": 49 }, { "name": "Hattie Toepperwein", "age": 22 }, { "name": "Melida Toepperwein", "age": null } ] }
+{ "cid": 992, "name": "Staci Alexandropoul", "age": null, "address": null, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Casimira Alexandropoul", "age": null }, { "name": "Kena Alexandropoul", "age": 54 }, { "name": "Ellie Alexandropoul", "age": null }, { "name": "Ambrose Alexandropoul", "age": null } ] }
+{ "cid": 993, "name": "Shawn Irie", "age": null, "address": null, "interests": [ "Fishing", "Cigars" ], "children": [ { "name": "Tonette Irie", "age": null } ] }
+{ "cid": 994, "name": "Isa Gravelle", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lashonda Gravelle", "age": null }, { "name": "Carry Gravelle", "age": 58 } ] }
+{ "cid": 995, "name": "Kiersten Basila", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Norman Basila", "age": 17 }, { "name": "Reginia Basila", "age": null }, { "name": "Gilberto Basila", "age": null }, { "name": "Elvira Basila", "age": 49 } ] }
+{ "cid": 996, "name": "Elouise Wider", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Base Jumping" ], "children": [  ] }
+{ "cid": 997, "name": "Yesenia Gao", "age": 38, "address": { "number": 5990, "street": "View St.", "city": "Portland" }, "interests": [ "Computers", "Computers", "Puzzles", "Puzzles" ], "children": [ { "name": "Jared Gao", "age": 11 }, { "name": "Sang Gao", "age": null }, { "name": "Jeanne Gao", "age": 13 }, { "name": "Lavona Gao", "age": 23 } ] }
+{ "cid": 998, "name": "Barry Schmaus", "age": 65, "address": { "number": 4894, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Ma Schmaus", "age": 40 }, { "name": "Lashawn Schmaus", "age": 13 }, { "name": "Georgianne Schmaus", "age": 38 } ] }
+{ "cid": 999, "name": "Bo Chaim", "age": 59, "address": { "number": 8050, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Zandra Chaim", "age": 42 }, { "name": "Theda Chaim", "age": 14 }, { "name": "Sharika Chaim", "age": 22 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance.adm
new file mode 100644
index 0000000..6d89122
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-edit-distance.adm
@@ -0,0 +1,8 @@
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [  ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-jaccard.adm
new file mode 100644
index 0000000..71bb9d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-olist-jaccard.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ulist-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ulist-jaccard.adm
new file mode 100644
index 0000000..fd1b75e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-ulist-jaccard.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": {{ "Computers", "Wine", "Databases", "Walking" }}, "children": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-contains.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-contains.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-contains.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-jaccard.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/inverted-index-word-jaccard.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-conjunctive-open.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-conjunctive-open.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-conjunctive-open.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-conjunctive-open.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-conjunctive.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-conjunctive.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-conjunctive.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-conjunctive.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-open.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-open.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey-open.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey-open.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/orders-index-custkey.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/orders-index-custkey.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/range-search-open.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/range-search-open.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/range-search-open.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/range-search-open.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/range-search.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/range-search.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/range-search.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/range-search.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index-nullable.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-nullable.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index-nullable.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-nullable.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index-open.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-open.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index-open.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index-open.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/index/rtree-secondary-index.adm
rename to asterix-app/src/test/resources/runtimets/results/index-selection/rtree-secondary-index.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/abs0.adm b/asterix-app/src/test/resources/runtimets/results/numeric/abs0.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/abs0.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/abs1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/abs1.adm
new file mode 100644
index 0000000..263ae97
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/abs1.adm
@@ -0,0 +1 @@
+{ "f0": 20i8, "f1": 23i16, "f2": 29, "f3": 21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/abs2.adm b/asterix-app/src/test/resources/runtimets/results/numeric/abs2.adm
new file mode 100644
index 0000000..a4bcbe8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/abs2.adm
@@ -0,0 +1 @@
+{ "f0": 20.1f, "f1": 2.056E-29f, "f2": NaNf, "f3": Infinityf, "f4": Infinityf, "f5": 0.0f, "f6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/abs3.adm b/asterix-app/src/test/resources/runtimets/results/numeric/abs3.adm
new file mode 100644
index 0000000..4da3272
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/abs3.adm
@@ -0,0 +1 @@
+{ "d0": 20.1d, "d1": 2.056E-29d, "d2": NaNd, "d3": Infinityd, "d4": Infinityd, "d5": 0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/abs4.adm b/asterix-app/src/test/resources/runtimets/results/numeric/abs4.adm
new file mode 100644
index 0000000..f365181
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/abs4.adm
@@ -0,0 +1 @@
+{ "f0": 20i8, "f1": 1.11d, "f2": 12.9d, "f3": 1.11d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/ceiling0.adm b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling0.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling0.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/ceiling1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling1.adm
new file mode 100644
index 0000000..bf278a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling1.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -23i16, "f2": -29, "f3": -21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/ceiling2.adm b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling2.adm
new file mode 100644
index 0000000..96e5d3d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling2.adm
@@ -0,0 +1 @@
+{ "f0": 21.0f, "f1": -0.0f, "f2": NaNf, "f3": Infinityf, "f4": -Infinityf, "f5": -0.0f, "f6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/ceiling3.adm b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling3.adm
new file mode 100644
index 0000000..bf497d4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling3.adm
@@ -0,0 +1 @@
+{ "d0": 21.0d, "d1": -0.0d, "d2": NaNd, "d3": Infinityd, "d4": -Infinityd, "d5": -0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/ceiling4.adm b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling4.adm
new file mode 100644
index 0000000..dcc17c0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/ceiling4.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -1.0d, "f2": 13.0d, "f3": 2.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/floor0.adm b/asterix-app/src/test/resources/runtimets/results/numeric/floor0.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/floor0.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/floor1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/floor1.adm
new file mode 100644
index 0000000..bf278a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/floor1.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -23i16, "f2": -29, "f3": -21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/floor2.adm b/asterix-app/src/test/resources/runtimets/results/numeric/floor2.adm
new file mode 100644
index 0000000..078d1596
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/floor2.adm
@@ -0,0 +1 @@
+{ "f0": 20.0f, "f1": -1.0f, "f2": NaNf, "f3": Infinityf, "f4": -Infinityf, "f5": -0.0f, "f6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/floor3.adm b/asterix-app/src/test/resources/runtimets/results/numeric/floor3.adm
new file mode 100644
index 0000000..58bdd79
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/floor3.adm
@@ -0,0 +1 @@
+{ "d0": 20.0d, "d1": -1.0d, "d2": NaNd, "d3": Infinityd, "d4": -Infinityd, "d5": -0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/floor4.adm b/asterix-app/src/test/resources/runtimets/results/numeric/floor4.adm
new file mode 100644
index 0000000..3e972b3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/floor4.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -2.0d, "f2": 12.0d, "f3": 1.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even0.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even0.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even0.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even1.adm
new file mode 100644
index 0000000..bf278a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even1.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -23i16, "f2": -29, "f3": -21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even2.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even2.adm
new file mode 100644
index 0000000..f5f9e3f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even2.adm
@@ -0,0 +1 @@
+{ "f0": 0.0f, "f1": -20.0f, "f2": NaNf, "f3": Infinityf, "f4": -Infinityf, "f5": -0.0f, "f6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even20.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even20.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even20.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even21.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even21.adm
new file mode 100644
index 0000000..bf278a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even21.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -23i16, "f2": -29, "f3": -21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even22.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even22.adm
new file mode 100644
index 0000000..3f0cead
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even22.adm
@@ -0,0 +1 @@
+{ "d0": 0.56f, "d1": 0.32f, "d2": NaNf, "d3": Infinityf, "d4": -Infinityf, "d5": -0.0f, "d6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even23.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even23.adm
new file mode 100644
index 0000000..914a79e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even23.adm
@@ -0,0 +1 @@
+{ "d0": 0.56d, "d1": 0.32d, "d2": NaNd, "d3": Infinityd, "d4": -Infinityd, "d5": -0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even24.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even24.adm
new file mode 100644
index 0000000..956f0ac
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even24.adm
@@ -0,0 +1 @@
+{ "d0": 0.02d, "d1": 0.02d, "d2": 3567.81d, "d3": 0.0d, "d4": 35600.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even3.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even3.adm
new file mode 100644
index 0000000..60b7c52
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even3.adm
@@ -0,0 +1 @@
+{ "d0": 0.0d, "d1": -20.0d, "d2": NaNd, "d3": Infinityd, "d4": -Infinityd, "d5": -0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even4.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even4.adm
new file mode 100644
index 0000000..5620cde
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even4.adm
@@ -0,0 +1 @@
+{ "d0": 2.0d, "d1": 2.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even5.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even5.adm
new file mode 100644
index 0000000..a53f62f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round-half-to-even5.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -2.0d, "f2": 12.0d, "f3": 2.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round0.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round0.adm
new file mode 100644
index 0000000..61c5048
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round0.adm
@@ -0,0 +1 @@
+{ "f0": 0i8, "f1": 0i16, "f2": 0, "f3": 0i64, "f4": 0i8, "f5": 0i16, "f6": 0, "f7": 0i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round1.adm
new file mode 100644
index 0000000..bf278a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round1.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -23i16, "f2": -29, "f3": -21i64, "f4": 20i8, "f5": 22i16, "f6": 23, "f7": 27i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round2.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round2.adm
new file mode 100644
index 0000000..791c1da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round2.adm
@@ -0,0 +1 @@
+{ "f0": 20.0f, "f1": 0.0f, "f2": 0.0f, "f3": 2.14748365E9f, "f4": -2.14748365E9f, "f5": 0.0f, "f6": 0.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round3.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round3.adm
new file mode 100644
index 0000000..1b1936d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round3.adm
@@ -0,0 +1 @@
+{ "d0": 20.0d, "d1": 0.0d, "d2": 0.0d, "d3": 9.223372036854776E18d, "d4": -9.223372036854776E18d, "d5": 0.0d, "d6": 0.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/round4.adm b/asterix-app/src/test/resources/runtimets/results/numeric/round4.adm
new file mode 100644
index 0000000..ce70363
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/round4.adm
@@ -0,0 +1 @@
+{ "f0": -20i8, "f1": -1.0d, "f2": 13.0d, "f3": 1.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_01.adm
deleted file mode 100644
index bd48346..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_01.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ true, 3 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_02.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_02.adm
deleted file mode 100644
index a2f763f..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_02.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ false, 2147483647 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_03.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_03.adm
deleted file mode 100644
index bd48346..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_03.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ true, 3 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_04.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_04.adm
deleted file mode 100644
index a2f763f..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_04.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ false, 2147483647 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_ints.adm
new file mode 100644
index 0000000..9aff56c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_ints.adm
@@ -0,0 +1,4 @@
+[ true, 3 ]
+[ true, 3 ]
+[ false, 2147483647 ]
+[ false, 2147483647 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_strings.adm
new file mode 100644
index 0000000..9aff56c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-check_strings.adm
@@ -0,0 +1,4 @@
+[ true, 3 ]
+[ true, 3 ]
+[ false, 2147483647 ]
+[ false, 2147483647 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-list-is-filterable.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-list-is-filterable.adm
new file mode 100644
index 0000000..fa1dde6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-list-is-filterable.adm
@@ -0,0 +1,6 @@
+false
+false
+true
+true
+true
+false
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-string-is-filterable.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-string-is-filterable.adm
new file mode 100644
index 0000000..0b36002
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance-string-is-filterable.adm
@@ -0,0 +1,20 @@
+false
+false
+false
+false
+true
+true
+true
+true
+true
+true
+false
+false
+true
+true
+true
+true
+true
+true
+false
+false
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_01.adm
deleted file mode 100644
index 00750ed..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_01.adm
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_02.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_02.adm
deleted file mode 100644
index 00750ed..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_02.adm
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_ints.adm
new file mode 100644
index 0000000..a5c8806
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_ints.adm
@@ -0,0 +1,2 @@
+3
+3
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_strings.adm
new file mode 100644
index 0000000..a5c8806
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/edit-distance_strings.adm
@@ -0,0 +1,2 @@
+3
+3
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-edit-distance.adm b/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-edit-distance.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-edit-distance.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-similarity-jaccard.adm b/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-similarity-jaccard.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/fuzzyeq-similarity-jaccard.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/prefix-len-jaccard_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/prefix-len-jaccard.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/similarity/prefix-len-jaccard_01.adm
rename to asterix-app/src/test/resources/runtimets/results/similarity/prefix-len-jaccard.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_01.adm
deleted file mode 100644
index d3c43af..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_01.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ true, 0.7f ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_02.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_02.adm
deleted file mode 100644
index bab8fb3..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_02.adm
+++ /dev/null
@@ -1 +0,0 @@
-[ false, 0.0f ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_ints.adm
new file mode 100644
index 0000000..ad2ef2e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_ints.adm
@@ -0,0 +1,12 @@
+[ true, 0.0f ]
+[ true, 0.0f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.05f ]
+[ true, 0.05f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_query.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_query.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_query.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings.adm
new file mode 100644
index 0000000..93b588a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings.adm
@@ -0,0 +1,16 @@
+[ true, 0.0f ]
+[ true, 0.0f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.05f ]
+[ true, 0.05f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix-check_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix-check.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix-check_01.adm
rename to asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix-check.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix_01.adm
rename to asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-prefix.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_ints.adm
new file mode 100644
index 0000000..2a05d33
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_ints.adm
@@ -0,0 +1,8 @@
+[ true, 0.0f ]
+[ true, 0.0f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_query.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_query.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_query.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_strings.adm
new file mode 100644
index 0000000..ef4b0f9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted-check_strings.adm
@@ -0,0 +1,12 @@
+[ true, 0.0f ]
+[ true, 0.0f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
+[ true, 0.7f ]
+[ true, 0.7f ]
+[ false, 0.0f ]
+[ false, 0.0f ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_ints.adm
new file mode 100644
index 0000000..0577394
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_ints.adm
@@ -0,0 +1,4 @@
+0.0f
+0.0f
+0.7f
+0.7f
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_query.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_query.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_query.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_strings.adm
new file mode 100644
index 0000000..427ee87
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-sorted_strings.adm
@@ -0,0 +1,6 @@
+0.0f
+0.0f
+0.7f
+0.7f
+0.7f
+0.7f
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_01.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_01.adm
deleted file mode 100644
index 31c8d6d..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_01.adm
+++ /dev/null
@@ -1 +0,0 @@
-0.7f
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_02.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_02.adm
deleted file mode 100644
index 118f6bd..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_02.adm
+++ /dev/null
@@ -1 +0,0 @@
-0.0f
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_03.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_03.adm
deleted file mode 100644
index 118f6bd..0000000
--- a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_03.adm
+++ /dev/null
@@ -1 +0,0 @@
-0.0f
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_ints.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_ints.adm
new file mode 100644
index 0000000..427ee87
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_ints.adm
@@ -0,0 +1,6 @@
+0.0f
+0.0f
+0.7f
+0.7f
+0.7f
+0.7f
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_query.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_query.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_query.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_strings.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_strings.adm
new file mode 100644
index 0000000..b261912
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard_strings.adm
@@ -0,0 +1,8 @@
+0.0f
+0.0f
+0.7f
+0.7f
+0.7f
+0.7f
+0.7f
+0.7f
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string1.adm b/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string1.adm
new file mode 100644
index 0000000..fdbb861
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string1.adm
@@ -0,0 +1 @@
+{ "result1": "中文字符" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string2.adm b/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string2.adm
new file mode 100644
index 0000000..8ab1bd7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/codepoint-to-string2.adm
@@ -0,0 +1 @@
+{ "f1": "", "f2": "abc" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/concat1.adm b/asterix-app/src/test/resources/runtimets/results/string/concat1.adm
new file mode 100644
index 0000000..cea6e76
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/concat1.adm
@@ -0,0 +1 @@
+{ "result1": "aa25991bb31526", "result2": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/end-with1.adm b/asterix-app/src/test/resources/runtimets/results/string/end-with1.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/end-with1.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/end-with2.adm b/asterix-app/src/test/resources/runtimets/results/string/end-with2.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/end-with2.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/end-with3.adm b/asterix-app/src/test/resources/runtimets/results/string/end-with3.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/end-with3.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/end-with4.adm b/asterix-app/src/test/resources/runtimets/results/string/end-with4.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/end-with4.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/end-with5.adm b/asterix-app/src/test/resources/runtimets/results/string/end-with5.adm
new file mode 100644
index 0000000..c13d3dc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/end-with5.adm
@@ -0,0 +1 @@
+{ "f1": true, "f2": false, "f3": true, "f4": false, "f5": true, "f6": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/length.adm b/asterix-app/src/test/resources/runtimets/results/string/length.adm
new file mode 100644
index 0000000..45a976e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/length.adm
@@ -0,0 +1 @@
+{ "result1": 6, "result2": 0, "result3": null }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/lowercase.adm b/asterix-app/src/test/resources/runtimets/results/string/lowercase.adm
new file mode 100644
index 0000000..a20b9aa
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/lowercase.adm
@@ -0,0 +1 @@
+{ "result1": "hellow", "result2": "", "result3": null }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches1.adm b/asterix-app/src/test/resources/runtimets/results/string/matches1.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches1.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches2.adm b/asterix-app/src/test/resources/runtimets/results/string/matches2.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches2.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches21.adm b/asterix-app/src/test/resources/runtimets/results/string/matches21.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches21.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches22.adm b/asterix-app/src/test/resources/runtimets/results/string/matches22.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches22.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches23.adm b/asterix-app/src/test/resources/runtimets/results/string/matches23.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches23.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matches3.adm b/asterix-app/src/test/resources/runtimets/results/string/matches3.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matches3.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/matchesnull.adm b/asterix-app/src/test/resources/runtimets/results/string/matchesnull.adm
new file mode 100644
index 0000000..6126426
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/matchesnull.adm
@@ -0,0 +1 @@
+{ "result1": false, "result2": false, "result3": true, "result4": false, "result5": false, "result6": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/replace1.adm b/asterix-app/src/test/resources/runtimets/results/string/replace1.adm
new file mode 100644
index 0000000..5f992ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/replace1.adm
@@ -0,0 +1 @@
+{ "result1": "brcdbr", "result2": "abbraccaddabbra", "result3": "carted" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/replace2.adm b/asterix-app/src/test/resources/runtimets/results/string/replace2.adm
new file mode 100644
index 0000000..fa45140
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/replace2.adm
@@ -0,0 +1 @@
+{ "result1": "a*cada*" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/replace21.adm b/asterix-app/src/test/resources/runtimets/results/string/replace21.adm
new file mode 100644
index 0000000..9fd2e6a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/replace21.adm
@@ -0,0 +1 @@
+{ "result1": "abracadabra", "result2": "akkkcadakkk", "result3": "kkk" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/replace22.adm b/asterix-app/src/test/resources/runtimets/results/string/replace22.adm
new file mode 100644
index 0000000..21ca784
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/replace22.adm
@@ -0,0 +1 @@
+{ "result1": "abracadabra", "result2": "aXXXcadaXXX", "result3": null, "result4": "aXXXcadaXXX", "result5": "XXXaXXXbXXXrXXXaXXXcXXXaXXXdXXXaXXXbXXXrXXXaXXX", "result6": "acada", "result7": "acada", "result8": "XXXaXXXbXXXrXXXaXXXcXXXaXXXdXXXaXXXbXXXrXXXaXXX" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/replace3.adm b/asterix-app/src/test/resources/runtimets/results/string/replace3.adm
new file mode 100644
index 0000000..3882682
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/replace3.adm
@@ -0,0 +1 @@
+{ "result1": "*" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/start-with1.adm b/asterix-app/src/test/resources/runtimets/results/string/start-with1.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/start-with1.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/start-with2.adm b/asterix-app/src/test/resources/runtimets/results/string/start-with2.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/start-with2.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/start-with3.adm b/asterix-app/src/test/resources/runtimets/results/string/start-with3.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/start-with3.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/start-with4.adm b/asterix-app/src/test/resources/runtimets/results/string/start-with4.adm
new file mode 100644
index 0000000..c13d3dc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/start-with4.adm
@@ -0,0 +1 @@
+{ "f1": true, "f2": false, "f3": true, "f4": false, "f5": true, "f6": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/start-with5.adm b/asterix-app/src/test/resources/runtimets/results/string/start-with5.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/start-with5.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-concat1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-concat1.adm
new file mode 100644
index 0000000..a5becc3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-concat1.adm
@@ -0,0 +1 @@
+{ "result1": "aa25991bb31526" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-equal-true1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-equal-true1.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-equal-true1.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-equal1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-equal1.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-equal1.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-equal2.adm b/asterix-app/src/test/resources/runtimets/results/string/string-equal2.adm
new file mode 100644
index 0000000..538da31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-equal2.adm
@@ -0,0 +1 @@
+{ "result1": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-equal3.adm b/asterix-app/src/test/resources/runtimets/results/string/string-equal3.adm
new file mode 100644
index 0000000..ea9ca72
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-equal3.adm
@@ -0,0 +1 @@
+{ "result1": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-equal4.adm b/asterix-app/src/test/resources/runtimets/results/string/string-equal4.adm
new file mode 100644
index 0000000..71a9bb6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-equal4.adm
@@ -0,0 +1 @@
+{ "result1": true, "result3": false, "result4": false, "result5": true }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-join1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-join1.adm
new file mode 100644
index 0000000..7276381
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-join1.adm
@@ -0,0 +1 @@
+{ "result0": "aa::25991::bb::31526", "result1": "aa25991bb31526" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint.adm b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint.adm
new file mode 100644
index 0000000..c67c60e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint.adm
@@ -0,0 +1 @@
+{ "result1": [ 97, 98, 99, 100 ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint1.adm
new file mode 100644
index 0000000..46eca36
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint1.adm
@@ -0,0 +1 @@
+{ "result1": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-after-1.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-after-1.adm
new file mode 100644
index 0000000..197a7af
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-after-1.adm
@@ -0,0 +1 @@
+{ "result1": "low" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-after-2.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-after-2.adm
new file mode 100644
index 0000000..04393a4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-after-2.adm
@@ -0,0 +1 @@
+{ "result1": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-after-3.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-after-3.adm
new file mode 100644
index 0000000..04393a4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-after-3.adm
@@ -0,0 +1 @@
+{ "result1": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-after-4.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-after-4.adm
new file mode 100644
index 0000000..9406ef5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-after-4.adm
@@ -0,0 +1 @@
+{ "result1": "HEllow", "result2": "HEllow", "result3": "", "result4": "", "result5": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-before-1.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-before-1.adm
new file mode 100644
index 0000000..1c6a7d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-before-1.adm
@@ -0,0 +1 @@
+{ "result1": "HE" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-before-2.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-before-2.adm
new file mode 100644
index 0000000..04393a4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-before-2.adm
@@ -0,0 +1 @@
+{ "result1": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring-before-3.adm b/asterix-app/src/test/resources/runtimets/results/string/substring-before-3.adm
new file mode 100644
index 0000000..9ce64e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring-before-3.adm
@@ -0,0 +1 @@
+{ "result1": "", "result2": "", "result3": "", "result4": "", "result5": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring2-1.adm b/asterix-app/src/test/resources/runtimets/results/string/substring2-1.adm
new file mode 100644
index 0000000..411e803
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring2-1.adm
@@ -0,0 +1 @@
+{ "result1": "Ellow" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring2-2.adm b/asterix-app/src/test/resources/runtimets/results/string/substring2-2.adm
new file mode 100644
index 0000000..5b0c7b9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring2-2.adm
@@ -0,0 +1 @@
+{ "result1": "HEllow" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring2-3.adm b/asterix-app/src/test/resources/runtimets/results/string/substring2-3.adm
new file mode 100644
index 0000000..04393a4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring2-3.adm
@@ -0,0 +1 @@
+{ "result1": "" }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substring2-4.adm b/asterix-app/src/test/resources/runtimets/results/string/substring2-4.adm
new file mode 100644
index 0000000..5b0c7b9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/substring2-4.adm
@@ -0,0 +1 @@
+{ "result1": "HEllow" }
diff --git a/asterix-app/src/test/resources/runtimets/results/writers/serialized_01.adm b/asterix-app/src/test/resources/runtimets/results/writers/serialized_01.adm
index 7862e5c..f2aee15 100644
--- a/asterix-app/src/test/resources/runtimets/results/writers/serialized_01.adm
+++ b/asterix-app/src/test/resources/runtimets/results/writers/serialized_01.adm
Binary files differ
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/ILiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/ILiteral.java
deleted file mode 100644
index 31cb27e..0000000
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/ILiteral.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.aql.base;
-
-import java.io.Serializable;
-
-public interface ILiteral extends Serializable {
-    public enum Type {
-        STRING,
-        INTEGER,
-        NULL,
-        TRUE,
-        FALSE,
-        FLOAT,
-        DOUBLE
-    }
-
-    public Type getLiteralType();
-
-    public String getStringValue();
-
-}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Literal.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Literal.java
new file mode 100644
index 0000000..f6c779a
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/base/Literal.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.base;
+
+import java.io.Serializable;
+
+public abstract class Literal implements Serializable {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -6468144574890768345L;
+
+    public enum Type {
+        STRING,
+        INTEGER,
+        NULL,
+        TRUE,
+        FALSE,
+        FLOAT,
+        DOUBLE,
+        LONG
+    }
+
+    abstract public Object getValue();
+    
+    abstract public Type getLiteralType();
+
+    public String getStringValue() {
+        return getValue().toString();
+    }    
+    
+    @Override
+    public int hashCode() {
+        return getValue().hashCode();
+    }    
+
+    public boolean equals(Object obj) {
+        if (!(obj instanceof Literal)) {
+            return false;
+        }
+        Literal literal = (Literal)obj;
+        return getValue().equals(literal.getValue());
+    } 
+    
+    @Override
+    public String toString() {
+        return getStringValue();
+    }    
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateIndexStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateIndexStatement.java
index 399092e..ab75af5 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateIndexStatement.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateIndexStatement.java
@@ -18,9 +18,20 @@
     private IndexType indexType = IndexType.BTREE;
     private boolean ifNotExists;
 
+    // Specific to NGram indexes.
+    private int gramLength;
+
     public CreateIndexStatement() {
     }
 
+    public void setGramLength(int gramLength) {
+        this.gramLength = gramLength;
+    }
+
+    public int getGramLength() {
+        return gramLength;
+    }
+
     public void setNeedToCreate(boolean needToCreate) {
         this.needToCreate = needToCreate;
     }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/LiteralExpr.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/LiteralExpr.java
index ba94cfc..496b0e0 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/LiteralExpr.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/LiteralExpr.java
@@ -1,26 +1,26 @@
 package edu.uci.ics.asterix.aql.expression;
 
 import edu.uci.ics.asterix.aql.base.Expression;
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 
 public class LiteralExpr implements Expression {
-    private ILiteral value;
+    private Literal value;
 
     public LiteralExpr() {
     }
 
-    public LiteralExpr(ILiteral value) {
+    public LiteralExpr(Literal value) {
         this.value = value;
     }
 
-    public ILiteral getValue() {
+    public Literal getValue() {
         return value;
     }
 
-    public void setValue(ILiteral value) {
+    public void setValue(Literal value) {
         this.value = value;
     }
 
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
index f01991b..1c681b1 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/visitor/AQLPrintVisitor.java
@@ -6,7 +6,7 @@
 
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 import edu.uci.ics.asterix.aql.base.Statement;
 import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
@@ -103,9 +103,9 @@
 
     @Override
     public void visit(LiteralExpr l, Integer step) {
-        ILiteral lc = l.getValue();
-        if (lc.getLiteralType().equals(ILiteral.Type.TRUE) || lc.getLiteralType().equals(ILiteral.Type.FALSE)
-                || lc.getLiteralType().equals(ILiteral.Type.NULL)) {
+    	Literal lc = l.getValue();
+        if (lc.getLiteralType().equals(Literal.Type.TRUE) || lc.getLiteralType().equals(Literal.Type.FALSE)
+                || lc.getLiteralType().equals(Literal.Type.NULL)) {
             out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "]");
         } else {
             out.println(skip(step) + "LiteralExpr [" + l.getValue().getLiteralType() + "] ["
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/DoubleLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/DoubleLiteral.java
index d35f5f3..fb0c827 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/DoubleLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/DoubleLiteral.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class DoubleLiteral implements ILiteral {
+public class DoubleLiteral extends Literal {
     /**
      * 
      */
@@ -28,10 +28,12 @@
         this.value = value;
     }
 
+    @Override    
     public Double getValue() {
         return value;
     }
 
+
     public void setValue(Double value) {
         this.value = value;
     }
@@ -40,28 +42,4 @@
     public Type getLiteralType() {
         return Type.DOUBLE;
     }
-
-    @Override
-    public String getStringValue() {
-        return value.toString();
-    }
-
-    @Override
-    public String toString() {
-        return getStringValue();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof DoubleLiteral)) {
-            return false;
-        }
-        DoubleLiteral d = (DoubleLiteral) obj;
-        return d.getValue() == value;
-    }
-
-    @Override
-    public int hashCode() {
-        return value.hashCode();
-    }
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FalseLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FalseLiteral.java
index 17b4254..7f19b84 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FalseLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FalseLiteral.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class FalseLiteral implements ILiteral {
+public class FalseLiteral extends Literal {
 
     private static final long serialVersionUID = -750814844423165149L;
 
@@ -49,4 +49,9 @@
     public int hashCode() {
         return (int) serialVersionUID;
     }
+
+    @Override
+    public Boolean getValue() {
+        return Boolean.FALSE;
+    }
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FloatLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FloatLiteral.java
index f77ed39..b2db6a1 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FloatLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/FloatLiteral.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class FloatLiteral implements ILiteral {
+public class FloatLiteral extends Literal {
     /**
      * 
      */
@@ -28,10 +28,12 @@
         this.value = value;
     }
 
+    @Override
     public Float getValue() {
         return value;
     }
 
+    
     public void setValue(Float value) {
         this.value = value;
     }
@@ -40,28 +42,4 @@
     public Type getLiteralType() {
         return Type.FLOAT;
     }
-
-    @Override
-    public String getStringValue() {
-        return value.toString();
-    }
-
-    @Override
-    public String toString() {
-        return getStringValue();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof FloatLiteral)) {
-            return false;
-        }
-        FloatLiteral f = (FloatLiteral) obj;
-        return f.getValue() == value;
-    }
-
-    @Override
-    public int hashCode() {
-        return value.hashCode();
-    }
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/IntegerLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/IntegerLiteral.java
index 52cf7fa..f29feb8 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/IntegerLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/IntegerLiteral.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class IntegerLiteral implements ILiteral {
+public class IntegerLiteral extends Literal {
     /**
      * 
      */
@@ -37,32 +37,7 @@
     }
 
     @Override
-    public String toString() {
-        return getStringValue();
-    }
-
-    @Override
     public Type getLiteralType() {
         return Type.INTEGER;
     }
-
-    @Override
-    public String getStringValue() {
-        return value.toString();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof IntegerLiteral)) {
-            return false;
-        }
-        IntegerLiteral i = (IntegerLiteral) obj;
-        return value.equals(i.getValue());
-    }
-
-    @Override
-    public int hashCode() {
-        return value;
-    }
-
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/LongIntegerLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/LongIntegerLiteral.java
new file mode 100644
index 0000000..7e02f04
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/LongIntegerLiteral.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.aql.literal;
+
+import edu.uci.ics.asterix.aql.base.Literal;
+
+public class LongIntegerLiteral extends Literal {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -8633520244871361967L;
+    private Long value;
+
+    public LongIntegerLiteral(Long value) {
+        super();
+        this.value = value;
+    }
+
+    @Override
+    public Long getValue() {
+        return value;
+    }
+
+    public void setValue(Long value) {
+        this.value = value;
+    }
+
+    @Override
+    public Type getLiteralType() {
+        return Type.LONG;
+    }
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/NullLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/NullLiteral.java
index b421afa..efcb9c6 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/NullLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/NullLiteral.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class NullLiteral implements ILiteral {
+public class NullLiteral extends Literal {
 
     /**
      * 
@@ -39,11 +39,6 @@
     }
 
     @Override
-    public String toString() {
-        return getStringValue();
-    }
-
-    @Override
     public boolean equals(Object obj) {
         return obj == INSTANCE;
     }
@@ -52,4 +47,9 @@
     public int hashCode() {
         return (int) serialVersionUID;
     }
+
+    @Override
+    public Object getValue() {
+        return null;
+    }    
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/StringLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/StringLiteral.java
index ad5588f..0fd586c 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/StringLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/StringLiteral.java
@@ -1,8 +1,8 @@
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class StringLiteral implements ILiteral {
+public class StringLiteral extends Literal {
 
     private static final long serialVersionUID = -6342491706277606168L;
     private String value;
@@ -29,24 +29,4 @@
     public String getStringValue() {
         return value;
     }
-
-    @Override
-    public String toString() {
-        return getStringValue();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof StringLiteral)) {
-            return false;
-        }
-        StringLiteral s = (StringLiteral) obj;
-        return value.equals(s.getValue());
-    }
-
-    @Override
-    public int hashCode() {
-        return value.hashCode();
-    }
-
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/TrueLiteral.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/TrueLiteral.java
index dedd52b..80d7966 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/TrueLiteral.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/literal/TrueLiteral.java
@@ -1,8 +1,8 @@
 package edu.uci.ics.asterix.aql.literal;
 
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 
-public class TrueLiteral implements ILiteral {
+public class TrueLiteral extends Literal {
     private static final long serialVersionUID = -8513245514578847512L;
 
     private TrueLiteral() {
@@ -34,4 +34,9 @@
     public int hashCode() {
         return (int) serialVersionUID;
     }
+
+    @Override
+    public Boolean getValue() {
+        return Boolean.TRUE;
+    }
 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
index fdb56be..8f0a3ff 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
@@ -83,7 +83,9 @@
     private String dataverseName;
 
     private enum DfsColor {
-        WHITE, GRAY, BLACK
+        WHITE,
+        GRAY,
+        BLACK
     }
 
     public AqlRewriter(Query topExpr, int varCounter, MetadataTransactionContext txnContext, String dataverseName) {
@@ -156,15 +158,15 @@
         List<AsterixFunction> functionCalls = getFunctionCalls(expression);
         for (AsterixFunction funId : functionCalls) {
             if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-                    funId.getFunctionName(), false))) {
+                    funId.getFunctionName()))) {
                 continue;
             }
 
-            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
-                    funId.getFunctionName(), false))) {
+            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(
+                    AlgebricksBuiltinFunctions.ALGEBRICKS_NS, funId.getFunctionName()))) {
                 continue;
             }
-            
+
             if (declaredFunctions != null && declaredFunctions.contains(funId)) {
                 continue;
             }
@@ -179,8 +181,8 @@
     }
 
     private FunctionDecl getFunctionDecl(AsterixFunction funId) throws AsterixException {
-        Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, dataverseName, funId.getFunctionName(), funId
-                .getArity());
+        Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, dataverseName, funId.getFunctionName(),
+                funId.getArity());
         if (function == null) {
             throw new AsterixException(" unknown function " + funId);
         }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
index d246a92..1a30693 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
@@ -77,19 +77,17 @@
     public static IFunctionInfo getFunctionInfo(MetadataTransactionContext mdTxnCtx, String dataverseName,
             AsterixFunction asterixFunction) throws MetadataException {
         FunctionIdentifier fid = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-                asterixFunction.getFunctionName(), asterixFunction.getArity(), true);
+                asterixFunction.getFunctionName(), asterixFunction.getArity());
         IFunctionInfo finfo = AsterixBuiltinFunctions.getAsterixFunctionInfo(fid);
         if (fid == null) {
             fid = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, asterixFunction.getFunctionName(),
-                    asterixFunction.getArity(), true);
+                    asterixFunction.getArity());
         }
         if (fid == null) {
-            Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, dataverseName, asterixFunction
-                    .getFunctionName(), asterixFunction.getArity());
+            Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, dataverseName,
+                    asterixFunction.getFunctionName(), asterixFunction.getArity());
             if (function != null) {
-                finfo = new AsterixFunctionInfo(dataverseName, asterixFunction, false);
-                // todo: for external functions, we shall construct another kind of function info (that extends AsterixFunctionInfo)
-                // and has additional information.
+                finfo = new AsterixFunctionInfo(dataverseName, asterixFunction);
             }
         }
         return finfo; // could be null
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 1fa40b8..0767c4e 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -20,8 +20,9 @@
 import edu.uci.ics.asterix.aql.literal.FloatLiteral;
 import edu.uci.ics.asterix.aql.literal.DoubleLiteral;
 import edu.uci.ics.asterix.aql.literal.FalseLiteral;
-import edu.uci.ics.asterix.aql.base.ILiteral;
+import edu.uci.ics.asterix.aql.base.Literal;
 import edu.uci.ics.asterix.aql.literal.IntegerLiteral;
+import edu.uci.ics.asterix.aql.literal.LongIntegerLiteral;
 import edu.uci.ics.asterix.aql.literal.NullLiteral;
 import edu.uci.ics.asterix.aql.literal.StringLiteral;
 import edu.uci.ics.asterix.aql.literal.TrueLiteral;
@@ -368,10 +369,18 @@
   <RIGHTPAREN>
     ("type"
   		("btree" { cis.setIndexType(IndexType.BTREE); }
-  		| "keyword" { cis.setIndexType(IndexType.KEYWORD); }
-  		| "qgram" { cis.setIndexType(IndexType.QGRAM); } 
-  		| "rtree" { cis.setIndexType(IndexType.RTREE); }   		  		  
-		)  
+  		| "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }  		
+  		| "rtree" { cis.setIndexType(IndexType.RTREE); }
+  		| "ngram"
+  		  <LEFTPAREN>
+  		  (<INTEGER_LITERAL>
+  		    {
+  		      cis.setIndexType(IndexType.NGRAM_INVIX);
+  		      cis.setGramLength(Integer.valueOf(token.image));
+  		    }
+  		  )
+  		  <RIGHTPAREN>	  		  
+		)
   	";"  	
   	| ";"
     )
@@ -1470,10 +1479,11 @@
 	{
 		if(expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
 		{
-			ILiteral lit = ((LiteralExpr)expr).getValue();
-			if(lit.getLiteralType() == ILiteral.Type.INTEGER) {
+			Literal lit = ((LiteralExpr)expr).getValue();
+			if(lit.getLiteralType() == Literal.Type.INTEGER || 
+			   lit.getLiteralType() == Literal.Type.LONG) {
 				idx = Integer.valueOf(lit.getStringValue());
-			}	
+			}
 			else {
 				throw new ParseException("Index should be an INTEGER");				
             }
@@ -1501,7 +1511,7 @@
   Expression expr = null;
 }
 {
-  //ILiteral | VariableRef | ListConstructor | RecordConstructor | FunctionCallExpr | ParenthesizedExpression
+  //Literal | VariableRef | ListConstructor | RecordConstructor | FunctionCallExpr | ParenthesizedExpression
 	(
 	  expr =Literal() 
 	   | expr = FunctionCallExpr()
@@ -1537,7 +1547,11 @@
  	 | <INTEGER_LITERAL>
     {
       t= getToken(0);
-	  lit.setValue(new IntegerLiteral(new Integer(t.image)));
+	  try {
+	      lit.setValue(new IntegerLiteral(new Integer(t.image)));
+	  } catch(NumberFormatException ex) {
+	      lit.setValue(new LongIntegerLiteral(new Long(t.image)));
+	  }
 	}
      | < FLOAT_LITERAL >
     {
@@ -1851,12 +1865,9 @@
 	extendCurrentScope();
 }
 {
-    "let" varExp = Variable()
+    "let" varExp = Variable() ":=" beExp = Expression()
     {
       getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
-	}
-     ":=" beExp = Expression()
-    {
       lc.setVarExpr(varExp);
       lc.setBeExpr(beExp);
       return lc;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
index 7102288..144a8824 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
@@ -3,8 +3,8 @@
 import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
+import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index aff5dd2..e860a0a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -38,10 +38,10 @@
     }
 
     public enum IndexType {
-        BTREE,
-        KEYWORD,
-        QGRAM,
-        RTREE
+        BTREE,        
+        RTREE,
+        WORD_INVIX,
+        NGRAM_INVIX
     }
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
index 36e8a4c..d676cb5 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class AsterixAppRuntimeContext {
+    private static final int DEFAULT_BUFFER_CACHE_PAGE_SIZE = 32768;
     private final INCApplicationContext ncApplicationContext;
     
     private IndexRegistry<IIndex> indexRegistry;
@@ -52,7 +53,7 @@
     }
 
     private int getBufferCachePageSize() {
-        int pageSize = ncApplicationContext.getRootContext().getFrameSize();
+        int pageSize = DEFAULT_BUFFER_CACHE_PAGE_SIZE;
         String pageSizeStr = System.getProperty(GlobalConfig.BUFFER_CACHE_PAGE_SIZE_PROPERTY, null);
         if (pageSizeStr != null) {
             try {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionArgumentsConstants.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionArgumentsConstants.java
deleted file mode 100644
index 2a0ed70..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionArgumentsConstants.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.asterix.common.functions;
-
-public interface FunctionArgumentsConstants {
-    public static final String BTREE_INDEX = "btree";
-
-    public static final String RTREE_INDEX = "rtree";
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
index 332b48b..e189df3 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
@@ -16,7 +16,6 @@
 
 import java.util.Map;
 
-import edu.uci.ics.asterix.external.data.adapter.api.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.data.adapter.api.IDatasourceReadAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
@@ -26,7 +25,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -37,7 +35,7 @@
     private final String adapter;
     private final Map<String, String> adapterConfiguration;
     private final IAType atype;
-    private  IDatasourceReadAdapter datasourceReadAdapter;
+    private IDatasourceReadAdapter datasourceReadAdapter;
 
     public ExternalDataScanOperatorDescriptor(JobSpecification spec, String adapter, Map<String, String> arguments,
             IAType atype, RecordDescriptor rDesc) {
@@ -49,9 +47,8 @@
     }
 
     @Override
-    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
-            ICCApplicationContext appCtx) {
-        
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCApplicationContext appCtx) {
+
         /*
         Comment: The following code is commented out. This is because constraints are being set at compile time so that they can 
         be propagated to upstream Asterix operators. Hyracks has to provide a way to propagate constraints to upstream operators.
@@ -106,8 +103,8 @@
             }
         };
     }
-    
-    public void setDatasourceAdapter(IDatasourceReadAdapter adapterInstance){
+
+    public void setDatasourceAdapter(IDatasourceReadAdapter adapterInstance) {
         this.datasourceReadAdapter = adapterInstance;
     }
 }
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 7d1f64f..f96d3da 100644
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -59,7 +59,7 @@
             IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
             IOperationCallbackProvider opCallbackProvider, long transactionId) {
         super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, tupleFilterFactory, opCallbackProvider);
+                comparatorFactories, dataflowHelperFactory, tupleFilterFactory, false, opCallbackProvider);
         this.fieldPermutation = fieldPermutation;
         this.op = op;
         this.transactionId = transactionId;
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index af4eefc..1fd8fee 100644
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -47,6 +47,7 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 
 public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
     private FrameTupleAccessor accessor;
     private TreeIndexDataflowHelper treeIndexHelper;
     private final IRecordDescriptorProvider recordDescProvider;
@@ -69,6 +70,7 @@
     public TreeIndexInsertUpdateDeleteOperatorNodePushable(TransactionContext txnContext,
             AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
             IRecordDescriptorProvider recordDescProvider, IndexOp op) {
+        this.ctx = ctx;
         treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.recordDescProvider = recordDescProvider;
@@ -99,7 +101,7 @@
     public void open() throws HyracksDataException {
         AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
                 .getOperatorDescriptor();
-        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
         writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
         writer.open();
@@ -109,7 +111,7 @@
             indexAccessor = treeIndex.createAccessor();
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {
-                tupleFilter = tupleFilterFactory.createTupleFilter();
+                tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
                 frameTuple = new FrameTupleReference();
             }
             initializeTransactionSupport();
@@ -128,7 +130,7 @@
         byte[] resourceId = DataUtil.intToByteArray(fileId);
         int tupleCount = accessor.getTupleCount();
         try {
-            for (int i = 0; i < tupleCount; i++) {                
+            for (int i = 0; i < tupleCount; i++) {
                 if (tupleFilter != null) {
                     frameTuple.reset(accessor, i);
                     if (!tupleFilter.accept(frameTuple)) {
@@ -194,9 +196,9 @@
     @Override
     public void fail() throws HyracksDataException {
         try {
-        	writer.fail();
+            writer.fail();
         } finally {
-        	txnContext.addCloseableResource(new ICloseable() {
+            txnContext.addCloseableResource(new ICloseable() {
                 @Override
                 public void close(TransactionContext txnContext) throws ACIDException {
                     try {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
index 65a6d8e..e205acf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
@@ -35,267 +35,259 @@
  * simply ignored, i.e., updates are not not applied to the cache.
  */
 public class MetadataCache {
-	// Key is dataverse name.
-	protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
-	// Key is dataverse name. Key of value map is dataset name.
-	protected final Map<String, Map<String, Dataset>> datasets = new HashMap<String, Map<String, Dataset>>();
-	// Key is dataverse name. Key of value map is datatype name.
-	protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<String, Map<String, Datatype>>();
-	// Key is dataverse name.
-	protected final Map<String, NodeGroup> nodeGroups = new HashMap<String, NodeGroup>();
-	// Key is function Identifier . Key of value map is function name.
-	protected final Map<FunctionIdentifier, Function> functions = new HashMap<FunctionIdentifier, Function>();
+    // Key is dataverse name.
+    protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
+    // Key is dataverse name. Key of value map is dataset name.
+    protected final Map<String, Map<String, Dataset>> datasets = new HashMap<String, Map<String, Dataset>>();
+    // Key is dataverse name. Key of value map is datatype name.
+    protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<String, Map<String, Datatype>>();
+    // Key is dataverse name.
+    protected final Map<String, NodeGroup> nodeGroups = new HashMap<String, NodeGroup>();
+    // Key is function Identifier . Key of value map is function name.
+    protected final Map<FunctionIdentifier, Function> functions = new HashMap<FunctionIdentifier, Function>();
 
-	// Atomically executes all metadata operations in ctx's log.
-	public void commit(MetadataTransactionContext ctx) {
-		// Forward roll the operations written in ctx's log.
-		int logIx = 0;
-		ArrayList<MetadataLogicalOperation> opLog = ctx.getOpLog();
-		try {
-			for (logIx = 0; logIx < opLog.size(); logIx++) {
-				doOperation(opLog.get(logIx));
-			}
-		} catch (Exception e) {
-			// Undo operations.
-			try {
-				for (int i = logIx - 1; i >= 0; i--) {
-					undoOperation(opLog.get(i));
-				}
-			} catch (Exception e2) {
-				// We encountered an error in undo. This case should never
-				// happen. Our only remedy to ensure cache consistency
-				// is to clear everything.
-				clear();
-			}
-		} finally {
-			ctx.clear();
-		}
-	}
+    // Atomically executes all metadata operations in ctx's log.
+    public void commit(MetadataTransactionContext ctx) {
+        // Forward roll the operations written in ctx's log.
+        int logIx = 0;
+        ArrayList<MetadataLogicalOperation> opLog = ctx.getOpLog();
+        try {
+            for (logIx = 0; logIx < opLog.size(); logIx++) {
+                doOperation(opLog.get(logIx));
+            }
+        } catch (Exception e) {
+            // Undo operations.
+            try {
+                for (int i = logIx - 1; i >= 0; i--) {
+                    undoOperation(opLog.get(i));
+                }
+            } catch (Exception e2) {
+                // We encountered an error in undo. This case should never
+                // happen. Our only remedy to ensure cache consistency
+                // is to clear everything.
+                clear();
+            }
+        } finally {
+            ctx.clear();
+        }
+    }
 
-	public void clear() {
-		synchronized (dataverses) {
-			synchronized (nodeGroups) {
-				synchronized (datasets) {
-					synchronized (datatypes) {
-						synchronized (functions) {
-							dataverses.clear();
-							nodeGroups.clear();
-							datasets.clear();
-							datatypes.clear();
-						}
-					}
-				}
-			}
-		}
-	}
+    public void clear() {
+        synchronized (dataverses) {
+            synchronized (nodeGroups) {
+                synchronized (datasets) {
+                    synchronized (datatypes) {
+                        synchronized (functions) {
+                            dataverses.clear();
+                            nodeGroups.clear();
+                            datasets.clear();
+                            datatypes.clear();
+                        }
+                    }
+                }
+            }
+        }
+    }
 
-	public Object addDataverseIfNotExists(Dataverse dataverse) {
-		synchronized (dataverses) {
-			synchronized (datasets) {
-				synchronized (datatypes) {
-					synchronized (functions) {
-						if (!dataverses.containsKey(dataverse)) {
-							datasets.put(dataverse.getDataverseName(),
-									new HashMap<String, Dataset>());
-							datatypes.put(dataverse.getDataverseName(),
-									new HashMap<String, Datatype>());
-							return dataverses.put(dataverse.getDataverseName(),
-									dataverse);
-						}
-					}
-					return null;
-				}
-			}
-		}
-	}
+    public Object addDataverseIfNotExists(Dataverse dataverse) {
+        synchronized (dataverses) {
+            synchronized (datasets) {
+                synchronized (datatypes) {
+                    synchronized (functions) {
+                        if (!dataverses.containsKey(dataverse)) {
+                            datasets.put(dataverse.getDataverseName(), new HashMap<String, Dataset>());
+                            datatypes.put(dataverse.getDataverseName(), new HashMap<String, Datatype>());
+                            return dataverses.put(dataverse.getDataverseName(), dataverse);
+                        }
+                    }
+                    return null;
+                }
+            }
+        }
+    }
 
-	public Object addDatasetIfNotExists(Dataset dataset) {
-		synchronized (datasets) {
-			Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
-			if (m == null) {
-				m = new HashMap<String, Dataset>();
-				datasets.put(dataset.getDataverseName(), m);
-			}
-			if (!m.containsKey(dataset.getDatasetName())) {
-				return m.put(dataset.getDatasetName(), dataset);
-			}
-			return null;
-		}
-	}
+    public Object addDatasetIfNotExists(Dataset dataset) {
+        synchronized (datasets) {
+            Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
+            if (m == null) {
+                m = new HashMap<String, Dataset>();
+                datasets.put(dataset.getDataverseName(), m);
+            }
+            if (!m.containsKey(dataset.getDatasetName())) {
+                return m.put(dataset.getDatasetName(), dataset);
+            }
+            return null;
+        }
+    }
 
-	public Object addDatatypeIfNotExists(Datatype datatype) {
-		synchronized (datatypes) {
-			Map<String, Datatype> m = datatypes
-					.get(datatype.getDataverseName());
-			if (m == null) {
-				m = new HashMap<String, Datatype>();
-				datatypes.put(datatype.getDataverseName(), m);
-			}
-			if (!m.containsKey(datatype.getDatatypeName())) {
-				return m.put(datatype.getDatatypeName(), datatype);
-			}
-			return null;
-		}
-	}
+    public Object addDatatypeIfNotExists(Datatype datatype) {
+        synchronized (datatypes) {
+            Map<String, Datatype> m = datatypes.get(datatype.getDataverseName());
+            if (m == null) {
+                m = new HashMap<String, Datatype>();
+                datatypes.put(datatype.getDataverseName(), m);
+            }
+            if (!m.containsKey(datatype.getDatatypeName())) {
+                return m.put(datatype.getDatatypeName(), datatype);
+            }
+            return null;
+        }
+    }
 
-	public Object addNodeGroupIfNotExists(NodeGroup nodeGroup) {
-		synchronized (nodeGroups) {
-			if (!nodeGroups.containsKey(nodeGroup.getNodeGroupName())) {
-				return nodeGroups.put(nodeGroup.getNodeGroupName(), nodeGroup);
-			}
-			return null;
-		}
-	}
+    public Object addNodeGroupIfNotExists(NodeGroup nodeGroup) {
+        synchronized (nodeGroups) {
+            if (!nodeGroups.containsKey(nodeGroup.getNodeGroupName())) {
+                return nodeGroups.put(nodeGroup.getNodeGroupName(), nodeGroup);
+            }
+            return null;
+        }
+    }
 
-	public Object dropDataverse(Dataverse dataverse) {
-		synchronized (dataverses) {
-			synchronized (datasets) {
-				synchronized (datatypes) {
-					synchronized (functions) {
-						datasets.remove(dataverse.getDataverseName());
-						datatypes.remove(dataverse.getDataverseName());
-						return dataverses.remove(dataverse.getDataverseName());
-					}
-				}
-			}
-		}
-	}
+    public Object dropDataverse(Dataverse dataverse) {
+        synchronized (dataverses) {
+            synchronized (datasets) {
+                synchronized (datatypes) {
+                    synchronized (functions) {
+                        datasets.remove(dataverse.getDataverseName());
+                        datatypes.remove(dataverse.getDataverseName());
+                        return dataverses.remove(dataverse.getDataverseName());
+                    }
+                }
+            }
+        }
+    }
 
-	public Object dropDataset(Dataset dataset) {
-		synchronized (datasets) {
-			Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
-			if (m == null) {
-				return null;
-			}
-			return m.remove(dataset.getDatasetName());
-		}
-	}
+    public Object dropDataset(Dataset dataset) {
+        synchronized (datasets) {
+            Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
+            if (m == null) {
+                return null;
+            }
+            return m.remove(dataset.getDatasetName());
+        }
+    }
 
-	public Object dropDatatype(Datatype datatype) {
-		synchronized (datatypes) {
-			Map<String, Datatype> m = datatypes
-					.get(datatype.getDataverseName());
-			if (m == null) {
-				return null;
-			}
-			return m.remove(datatype.getDatatypeName());
-		}
-	}
+    public Object dropDatatype(Datatype datatype) {
+        synchronized (datatypes) {
+            Map<String, Datatype> m = datatypes.get(datatype.getDataverseName());
+            if (m == null) {
+                return null;
+            }
+            return m.remove(datatype.getDatatypeName());
+        }
+    }
 
-	public Object dropNodeGroup(NodeGroup nodeGroup) {
-		synchronized (nodeGroups) {
-			return nodeGroups.remove(nodeGroup.getNodeGroupName());
-		}
-	}
+    public Object dropNodeGroup(NodeGroup nodeGroup) {
+        synchronized (nodeGroups) {
+            return nodeGroups.remove(nodeGroup.getNodeGroupName());
+        }
+    }
 
-	public Dataverse getDataverse(String dataverseName) {
-		synchronized (dataverses) {
-			return dataverses.get(dataverseName);
-		}
-	}
+    public Dataverse getDataverse(String dataverseName) {
+        synchronized (dataverses) {
+            return dataverses.get(dataverseName);
+        }
+    }
 
-	public Dataset getDataset(String dataverseName, String datasetName) {
-		synchronized (datasets) {
-			Map<String, Dataset> m = datasets.get(dataverseName);
-			if (m == null) {
-				return null;
-			}
-			return m.get(datasetName);
-		}
-	}
+    public Dataset getDataset(String dataverseName, String datasetName) {
+        synchronized (datasets) {
+            Map<String, Dataset> m = datasets.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.get(datasetName);
+        }
+    }
 
-	public Datatype getDatatype(String dataverseName, String datatypeName) {
-		synchronized (datatypes) {
-			Map<String, Datatype> m = datatypes.get(dataverseName);
-			if (m == null) {
-				return null;
-			}
-			return m.get(datatypeName);
-		}
-	}
+    public Datatype getDatatype(String dataverseName, String datatypeName) {
+        synchronized (datatypes) {
+            Map<String, Datatype> m = datatypes.get(dataverseName);
+            if (m == null) {
+                return null;
+            }
+            return m.get(datatypeName);
+        }
+    }
 
-	public NodeGroup getNodeGroup(String nodeGroupName) {
-		synchronized (nodeGroups) {
-			return nodeGroups.get(nodeGroupName);
-		}
-	}
+    public NodeGroup getNodeGroup(String nodeGroupName) {
+        synchronized (nodeGroups) {
+            return nodeGroups.get(nodeGroupName);
+        }
+    }
 
-	public Function getFunction(String dataverse, String functionName, int arity) {
-		synchronized (functions) {
-			return functions.get(new FunctionIdentifier(dataverse,
-					functionName, arity, false));
-		}
-	}
+    public Function getFunction(String dataverse, String functionName, int arity) {
+        synchronized (functions) {
+            return functions.get(new FunctionIdentifier(dataverse, functionName, arity));
+        }
+    }
 
-	public List<Dataset> getDataverseDatasets(String dataverseName) {
-		List<Dataset> retDatasets = new ArrayList<Dataset>();
-		synchronized (datasets) {
-			Map<String, Dataset> m = datasets.get(dataverseName);
-			if (m == null) {
-				return retDatasets;
-			}
-			for (Map.Entry<String, Dataset> entry : m.entrySet()) {
-				retDatasets.add(entry.getValue());
-			}
-			return retDatasets;
-		}
-	}
+    public List<Dataset> getDataverseDatasets(String dataverseName) {
+        List<Dataset> retDatasets = new ArrayList<Dataset>();
+        synchronized (datasets) {
+            Map<String, Dataset> m = datasets.get(dataverseName);
+            if (m == null) {
+                return retDatasets;
+            }
+            for (Map.Entry<String, Dataset> entry : m.entrySet()) {
+                retDatasets.add(entry.getValue());
+            }
+            return retDatasets;
+        }
+    }
 
-	/**
-	 * Represents a logical operation against the metadata.
-	 */
-	protected class MetadataLogicalOperation {
-		// Entity to be added/dropped.
-		public final IMetadataEntity entity;
-		// True for add, false for drop.
-		public final boolean isAdd;
+    /**
+     * Represents a logical operation against the metadata.
+     */
+    protected class MetadataLogicalOperation {
+        // Entity to be added/dropped.
+        public final IMetadataEntity entity;
+        // True for add, false for drop.
+        public final boolean isAdd;
 
-		public MetadataLogicalOperation(IMetadataEntity entity, boolean isAdd) {
-			this.entity = entity;
-			this.isAdd = isAdd;
-		}
-	};
+        public MetadataLogicalOperation(IMetadataEntity entity, boolean isAdd) {
+            this.entity = entity;
+            this.isAdd = isAdd;
+        }
+    };
 
-	protected void doOperation(MetadataLogicalOperation op) {
-		if (op.isAdd) {
-			op.entity.addToCache(this);
-		} else {
-			op.entity.dropFromCache(this);
-		}
-	}
+    protected void doOperation(MetadataLogicalOperation op) {
+        if (op.isAdd) {
+            op.entity.addToCache(this);
+        } else {
+            op.entity.dropFromCache(this);
+        }
+    }
 
-	protected void undoOperation(MetadataLogicalOperation op) {
-		if (!op.isAdd) {
-			op.entity.addToCache(this);
-		} else {
-			op.entity.dropFromCache(this);
-		}
-	}
+    protected void undoOperation(MetadataLogicalOperation op) {
+        if (!op.isAdd) {
+            op.entity.addToCache(this);
+        } else {
+            op.entity.dropFromCache(this);
+        }
+    }
 
-	public Object addFunctionIfNotExists(Function function) {
-		synchronized (functions) {
-			FunctionIdentifier fId = new FunctionIdentifier(
-					function.getDataverseName(), function.getFunctionName(),
-					function.getFunctionArity(), false);
+    public Object addFunctionIfNotExists(Function function) {
+        synchronized (functions) {
+            FunctionIdentifier fId = new FunctionIdentifier(function.getDataverseName(), function.getFunctionName(),
+                    function.getFunctionArity());
 
-			Function fun = functions.get(fId);
-			if (fun == null) {
-				return functions.put(fId, function);
-			}
-			return null;
-		}
-	}
+            Function fun = functions.get(fId);
+            if (fun == null) {
+                return functions.put(fId, function);
+            }
+            return null;
+        }
+    }
 
-	public Object dropFunction(Function function) {
-		synchronized (functions) {
-			FunctionIdentifier fId = new FunctionIdentifier(
-					function.getDataverseName(), function.getFunctionName(),
-					function.getFunctionArity(), false);
-			Function fun = functions.get(fId);
-			if (fun == null) {
-				return null;
-			}
-			return functions.remove(fId);
-		}
-	}
+    public Object dropFunction(Function function) {
+        synchronized (functions) {
+            FunctionIdentifier fId = new FunctionIdentifier(function.getDataverseName(), function.getFunctionName(),
+                    function.getFunctionArity());
+            Function fun = functions.get(fId);
+            if (fun == null) {
+                return null;
+            }
+            return functions.remove(fId);
+        }
+    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 850eceb..bd4f9bb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -18,7 +18,6 @@
 import java.rmi.RemoteException;
 import java.util.List;
 
-import edu.uci.ics.asterix.metadata.MetadataCache.MetadataLogicalOperation;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataManager;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
@@ -423,72 +422,65 @@
         }
         return nodeGroup;
     }
-   
 
-	
-	@Override
-	public void addFunction(MetadataTransactionContext mdTxnCtx,
-			Function function) throws MetadataException {
-		try {
-			metadataNode.addFunction(mdTxnCtx.getTxnId(), function);
-		} catch (RemoteException e) {
-			throw new MetadataException(e);
-		}
-		mdTxnCtx.addFunction(function);
-	}
+    @Override
+    public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException {
+        try {
+            metadataNode.addFunction(mdTxnCtx.getTxnId(), function);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        mdTxnCtx.addFunction(function);
+    }
 
-	@Override
-	public void dropFunction(MetadataTransactionContext ctx,
-			String dataverseName, String functionName, int arity)
-			throws MetadataException {
-		try {
-			metadataNode.dropFunction(ctx.getTxnId(), dataverseName,
-					functionName, arity);
-		} catch (RemoteException e) {
-			throw new MetadataException(e);
-		}
-		ctx.dropFunction(dataverseName, functionName, arity);
-	}
+    @Override
+    public void dropFunction(MetadataTransactionContext ctx, String dataverseName, String functionName, int arity)
+            throws MetadataException {
+        try {
+            metadataNode.dropFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        ctx.dropFunction(dataverseName, functionName, arity);
+    }
 
-	@Override
-	public Function getFunction(MetadataTransactionContext ctx,
-			String dataverseName, String functionName, int arity)
-			throws MetadataException {
-		// First look in the context to see if this transaction created the
-		// requested dataset itself (but the dataset is still uncommitted).
-		Function function = ctx.getFunction(dataverseName, functionName, arity);
-		if (function != null) {
-			// Don't add this dataverse to the cache, since it is still
-			// uncommitted.
-			return function;
-		}
-		if (ctx.functionIsDropped(dataverseName, functionName, arity)) {
-			// Dataset has been dropped by this transaction but could still be
-			// in the cache.
-			return null;
-		}
-		if (ctx.getDataverse(dataverseName) != null) {
-			// This transaction has dropped and subsequently created the same
-			// dataverse.
-			return null;
-		}
-		function = cache.getFunction(dataverseName, functionName, arity);
-		if (function != null) {
-			// Function is already in the cache, don't add it again.
-			return function;
-		}
-		try {
-			function = metadataNode.getFunction(ctx.getTxnId(), dataverseName,
-					functionName, arity);
-		} catch (RemoteException e) {
-			throw new MetadataException(e);
-		}
-		// We fetched the function from the MetadataNode. Add it to the cache
-		// when this transaction commits.
-		if (function != null) {
-			ctx.addFunction(function);
-		}
-		return function;
+    @Override
+    public Function getFunction(MetadataTransactionContext ctx, String dataverseName, String functionName, int arity)
+            throws MetadataException {
+        // First look in the context to see if this transaction created the
+        // requested dataset itself (but the dataset is still uncommitted).
+        Function function = ctx.getFunction(dataverseName, functionName, arity);
+        if (function != null) {
+            // Don't add this dataverse to the cache, since it is still
+            // uncommitted.
+            return function;
+        }
+        if (ctx.functionIsDropped(dataverseName, functionName, arity)) {
+            // Dataset has been dropped by this transaction but could still be
+            // in the cache.
+            return null;
+        }
+        if (ctx.getDataverse(dataverseName) != null) {
+            // This transaction has dropped and subsequently created the same
+            // dataverse.
+            return null;
+        }
+        function = cache.getFunction(dataverseName, functionName, arity);
+        if (function != null) {
+            // Function is already in the cache, don't add it again.
+            return function;
+        }
+        try {
+            function = metadataNode.getFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+        // We fetched the function from the MetadataNode. Add it to the cache
+        // when this transaction commits.
+        if (function != null) {
+            ctx.addFunction(function);
+        }
+        return function;
 
-	}
+    }
 }
\ No newline at end of file
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 6ac37c3..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();
@@ -152,7 +152,7 @@
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
             ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
             insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 // Add the primary index for the dataset.
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
@@ -163,7 +163,7 @@
                 insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
             }
             // Add entry in datatype secondary index.
-            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getDatatypeName(),
+            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
                     dataset.getDatasetName());
             insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
         } catch (BTreeDuplicateKeyException e) {
@@ -320,7 +320,7 @@
             ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             // Delete entry from secondary index 'group'.
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
                 // Searches the index for the tuple to be deleted. Acquires an S
@@ -330,14 +330,14 @@
                 deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
             }
             // Delete entry from secondary index 'type'.
-            ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getDatatypeName(), datasetName);
+            ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the DATATYPENAME_ON_DATASET_INDEX index.
             ITupleReference dataTypeTuple = getTupleToBeDeleted(txnId,
                     MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
             deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
             // Delete entry(s) from the 'indexes' dataset.
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
                 for (Index index : datasetIndexes) {
                     dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
deleted file mode 100644
index 1d806b1..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledDatasetDecl {
-
-    private String name;
-    private String itemTypeName;
-    private DatasetType datasetType;
-    private IAqlCompiledDatasetDetails acdd;
-
-    public AqlCompiledDatasetDecl(String name, String itemTypeName, DatasetType datasetType,
-            IAqlCompiledDatasetDetails acdd) {
-        this.name = name;
-        this.itemTypeName = itemTypeName;
-        this.datasetType = datasetType;
-        this.acdd = acdd;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getItemTypeName() {
-        return itemTypeName;
-    }
-
-    public DatasetType getDatasetType() {
-        return datasetType;
-    }
-
-    public IAqlCompiledDatasetDetails getAqlCompiledDatasetDetails() {
-        return acdd;
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
deleted file mode 100644
index 2e5eb3a..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledExternalDatasetDetails implements IAqlCompiledDatasetDetails {
-
-    private final String adapter;
-    private final Map<String, String> properties;
-
-    public AqlCompiledExternalDatasetDetails(String adapter, Map<String, String> properties) {
-        this.adapter = adapter;
-        this.properties = properties;
-    }
-
-    public String getAdapter() {
-        return adapter;
-    }
-
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    @Override
-    public DatasetType getDatasetType() {
-        return DatasetType.EXTERNAL;
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
deleted file mode 100644
index 5cfc8be..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledFeedDatasetDetails extends
-		AqlCompiledInternalDatasetDetails {
-	private final String adapter;
-	private final Map<String, String> properties;
-	private final String functionIdentifier;
-	private final String feedState;
-
-	public AqlCompiledFeedDatasetDetails(
-			List<String> partitioningExprs,
-			List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns,
-			String nodegroupName, AqlCompiledIndexDecl primaryIndex,
-			List<AqlCompiledIndexDecl> secondaryIndexes, String adapter,
-			Map<String, String> properties, String functionIdentifier,
-			String feedState) {
-		super(partitioningExprs, partitionFuns, nodegroupName, primaryIndex,
-				secondaryIndexes);
-		this.adapter = adapter;
-		this.properties = properties;
-		this.functionIdentifier = functionIdentifier;
-		this.feedState = feedState;
-	}
-
-	public String getAdapter() {
-		return adapter;
-	}
-
-	public Map<String, String> getProperties() {
-		return properties;
-	}
-
-	public String getFunctionIdentifier() {
-		return functionIdentifier;
-	}
-
-	public DatasetType getDatasetType() {
-		return DatasetType.FEED;
-	}
-
-	public String getFeedState() {
-		return feedState;
-	}
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
deleted file mode 100644
index 94718b0..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnionType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-
-public class AqlCompiledIndexDecl {
-
-    public enum IndexKind {
-        BTREE,
-        RTREE
-    }
-
-    private String indexName;
-    private IndexKind kind;
-    private List<String> fieldExprs = new ArrayList<String>();
-
-    public AqlCompiledIndexDecl(String indexName, IndexKind kind, List<String> fieldExprs) {
-        this.indexName = indexName;
-        this.kind = kind;
-        this.fieldExprs = fieldExprs;
-    }
-
-    @Override
-    public String toString() {
-        return "INDEX " + indexName + " (" + kind + ") " + fieldExprs;
-    }
-
-    public IndexKind getKind() {
-        return kind;
-    }
-
-    public String getIndexName() {
-        return indexName;
-    }
-
-    public List<String> getFieldExprs() {
-        return fieldExprs;
-    }
-
-    public static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
-        String[] names = recType.getFieldNames();
-        int n = names.length;
-        for (int i = 0; i < n; i++) {
-            if (names[i].equals(expr)) {
-                return recType.getFieldTypes()[i];
-            }
-        }
-        throw new AlgebricksException("Could not find field " + expr + " in the schema.");
-    }
-
-    public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType) throws AlgebricksException {
-        IAType keyType = AqlCompiledIndexDecl.keyFieldType(expr, recType);
-        boolean nullable = false;
-        if (keyType.getTypeTag() == ATypeTag.UNION) {
-            AUnionType unionType = (AUnionType) keyType;
-            if (unionType.isNullableType()) {
-                // The non-null type is always at index 1.
-                keyType = unionType.getUnionList().get(1);
-                nullable = true;
-            }
-        }
-        return new Pair<IAType, Boolean>(keyType, nullable);
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
deleted file mode 100644
index 912b766..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledInternalDatasetDetails implements IAqlCompiledDatasetDetails {
-    private final List<String> partitioningExprs;
-    private final List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns;
-    private final String nodegroupName;
-    private final List<AqlCompiledIndexDecl> secondaryIndexes;
-    private final AqlCompiledIndexDecl primaryIndex;
-    private HashMap<String, List<AqlCompiledIndexDecl>> secondaryIndexInvertedList;
-
-    public AqlCompiledInternalDatasetDetails(List<String> partitioningExprs,
-            List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns, String nodegroupName,
-            AqlCompiledIndexDecl primaryIndex, List<AqlCompiledIndexDecl> secondaryIndexes) {
-        this.partitioningExprs = partitioningExprs;
-        this.partitionFuns = partitionFuns;
-        this.nodegroupName = nodegroupName;
-        this.primaryIndex = primaryIndex;
-        this.secondaryIndexes = secondaryIndexes;
-        invertSecondaryIndexExprs();
-    }
-
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("dataset partitioned-by " + partitionFuns + " on " + nodegroupName);
-        if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
-            sb.append(System.getProperty("line.separator") + " with indexes: " + secondaryIndexes);
-        }
-        return sb.toString();
-    }
-
-    public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions() {
-        return partitionFuns;
-    }
-
-    public List<String> getPartitioningExprs() {
-        return partitioningExprs;
-    }
-
-    public int getPositionOfPartitioningKeyField(String fieldName) {
-        int pos = 0;
-        for (String pe : partitioningExprs) {
-            if (pe.equals(fieldName)) {
-                return pos;
-            }
-            ++pos;
-        }
-        return -1;
-    }
-
-    public String getNodegroupName() {
-        return nodegroupName;
-    }
-
-    public List<AqlCompiledIndexDecl> getSecondaryIndexes() {
-        return secondaryIndexes;
-    }
-
-    public AqlCompiledIndexDecl getPrimaryIndex() {
-        return primaryIndex;
-    }
-
-    public List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(String fieldExpr) {
-        return secondaryIndexInvertedList.get(fieldExpr);
-    }
-
-    public AqlCompiledIndexDecl findSecondaryIndexByExactKeyList(List<String> fieldExprs) {
-        if (secondaryIndexes == null) {
-            return null;
-        }
-        for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-            if (acid.getFieldExprs().equals(fieldExprs)) {
-                return acid;
-            }
-        }
-        return null;
-    }
-
-    public AqlCompiledIndexDecl findSecondaryIndexByName(String idxName) {
-        if (secondaryIndexes == null) {
-            return null;
-        }
-        for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-            if (acid.getIndexName().equals(idxName)) {
-                return acid;
-            }
-        }
-        return null;
-    }
-
-    private void invertSecondaryIndexExprs() {
-        secondaryIndexInvertedList = new HashMap<String, List<AqlCompiledIndexDecl>>();
-        if (secondaryIndexes == null) {
-            return;
-        }
-        for (AqlCompiledIndexDecl idx : secondaryIndexes) {
-            for (String s : idx.getFieldExprs()) {
-                List<AqlCompiledIndexDecl> idxList = secondaryIndexInvertedList.get(s);
-                if (idxList == null) {
-                    idxList = new ArrayList<AqlCompiledIndexDecl>();
-                    secondaryIndexInvertedList.put(s, idxList);
-                }
-                idxList.add(idx);
-            }
-        }
-    }
-
-    @Override
-    public DatasetType getDatasetType() {
-        return DatasetType.INTERNAL;
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index c76208c..ba7c797 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -23,7 +23,6 @@
 
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -31,25 +30,18 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.api.IMetadataManager;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -81,10 +73,11 @@
         this.dataverseName = dataverseName;
         this.outputFile = outputFile;
         this.config = config;
-        if (stores == null && online)
+        if (stores == null && online) {
             this.stores = AsterixProperties.INSTANCE.getStores();
-        else
+        } else {
             this.stores = stores;
+        }
         this.types = types;
         this.typeDataGenMap = typeDataGenMap;
         this.writerFactory = writerFactory;
@@ -113,17 +106,16 @@
     }
 
     public void disconnectFromDataverse() throws AlgebricksException {
-        if (!isConnected)
+        if (!isConnected) {
             throw new AlgebricksException("You are not connected to any dataverse");
-        else {
-            dataverseName = null;
-            format = null;
-            isConnected = false;
         }
+        dataverseName = null;
+        format = null;
+        isConnected = false;
     }
 
     public boolean isConnectedToDataverse() {
-        return this.isConnected;
+        return isConnected;
     }
 
     public String getDataverseName() {
@@ -135,8 +127,9 @@
     }
 
     public IDataFormat getFormat() throws AlgebricksException {
-        if (!isConnected)
+        if (!isConnected) {
             throw new AlgebricksException("You need first to connect to a dataverse.");
+        }
         return format;
     }
 
@@ -157,8 +150,13 @@
         return type.getDatatype();
     }
 
-    public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException, MetadataException {
-        NodeGroup ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+    public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException {
+        NodeGroup ng;
+        try {
+            ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
         if (ng == null) {
             throw new AlgebricksException("No node group with this name " + nodeGroupName);
         }
@@ -173,90 +171,44 @@
         return stores;
     }
 
-    public AqlCompiledDatasetDecl findDataset(String datasetName) {
+    public Dataset findDataset(String datasetName) throws AlgebricksException {
         try {
-            Dataset datasetRecord = this.metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
-            if (datasetRecord == null) {
-                return null;
-            }
-
-            IAqlCompiledDatasetDetails acdd = null;
-            switch (datasetRecord.getType()) {
-                case FEED:
-                case INTERNAL: {
-                    String typeName = datasetRecord.getDatatypeName();
-                    InternalDatasetDetails id = (InternalDatasetDetails) datasetRecord.getDatasetDetails();
-                    ARecordType recType = (ARecordType) findType(typeName);
-                    List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningEvalFactories = computePartitioningEvaluatorFactories(
-                            id.getPartitioningKey(), recType);
-                    List<Index> indexRecord = this.metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
-                    AqlCompiledIndexDecl primaryIndex = null;
-                    List<AqlCompiledIndexDecl> secondaryIndexes = new ArrayList<AqlCompiledIndexDecl>();
-                    for (int i = 0; i < indexRecord.size(); i++) {
-                        Index rec = indexRecord.get(i);
-                        if (rec.isPrimaryIndex()) {
-                            primaryIndex = new AqlCompiledIndexDecl(rec.getIndexName(), IndexKind.BTREE,
-                                    rec.getKeyFieldNames());
-                        } else {
-                            secondaryIndexes.add(new AqlCompiledIndexDecl(rec.getIndexName(),
-                                    rec.getIndexType() == IndexType.BTREE ? IndexKind.BTREE : IndexKind.RTREE, rec
-                                            .getKeyFieldNames()));
-                        }
-                    }
-
-                    if (datasetRecord.getType() == DatasetType.INTERNAL) {
-                        acdd = new AqlCompiledInternalDatasetDetails(id.getPartitioningKey(),
-                                partitioningEvalFactories, id.getNodeGroupName(), primaryIndex, secondaryIndexes);
-                    } else {
-                        acdd = new AqlCompiledFeedDatasetDetails(id.getPartitioningKey(), partitioningEvalFactories,
-                                id.getNodeGroupName(), primaryIndex, secondaryIndexes,
-                                ((FeedDatasetDetails) id).getAdapter(), ((FeedDatasetDetails) id).getProperties(),
-                                ((FeedDatasetDetails) id).getFunctionIdentifier(), ((FeedDatasetDetails) id)
-                                        .getFeedState().toString());
-                    }
-                    break;
-                }
-
-                case EXTERNAL: {
-                    acdd = new AqlCompiledExternalDatasetDetails(
-                            ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getAdapter(),
-                            ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getProperties());
-                    break;
-                }
-
-            }
-            AqlCompiledDatasetDecl dataset = new AqlCompiledDatasetDecl(datasetRecord.getDatasetName(),
-                    datasetRecord.getDatatypeName(), datasetRecord.getType(), acdd);
-            return dataset;
-
-        } catch (Exception e) {
-            throw new IllegalStateException(e);
+            return metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
         }
     }
 
+    public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
+    	try {
+            return metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+    
+    public Index getDatasetPrimaryIndex(String dataverseName, String datasetName) throws AlgebricksException {
+        try {
+            return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    public Index getIndex(String dataverseName, String datasetName, String indexName) throws AlgebricksException {
+        try {
+            return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+    
     public void setOutputFile(FileSplit outputFile) {
         this.outputFile = outputFile;
     }
 
-    public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> computePartitioningEvaluatorFactories(
-            List<String> partitioningExprs, ARecordType recType) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> evalFactories = new ArrayList<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>>(
-                partitioningExprs.size());
-        for (String expr : partitioningExprs) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFact = null;
-            try {
-                evalFact = format.partitioningEvaluatorFactory(recType, expr);
-            } catch (AlgebricksException e) {
-                throw new IllegalStateException(e);
-            }
-            evalFactories.add(evalFact);
-        }
-        return evalFactories;
-    }
-
-	public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
-            String datasetName, String targetIdxName) throws AlgebricksException, MetadataException {
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+            String datasetName, String targetIdxName) throws AlgebricksException {
         FileSplit[] splits = splitsForInternalOrFeedDataset(datasetName, targetIdxName);
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         String[] loc = new String[splits.length];
@@ -267,24 +219,36 @@
         return new Pair<IFileSplitProvider, AlgebricksPartitionConstraint>(splitProvider, pc);
     }
 
+    public Pair<IFileSplitProvider, IFileSplitProvider> getInvertedIndexFileSplitProviders(
+            IFileSplitProvider splitProvider) {
+        int numSplits = splitProvider.getFileSplits().length;
+        FileSplit[] btreeSplits = new FileSplit[numSplits];
+        FileSplit[] invListsSplits = new FileSplit[numSplits];
+        for (int i = 0; i < numSplits; i++) {
+            String nodeName = splitProvider.getFileSplits()[i].getNodeName();
+            String path = splitProvider.getFileSplits()[i].getLocalFile().getFile().getPath();
+            btreeSplits[i] = new FileSplit(nodeName, path + "_$btree");
+            invListsSplits[i] = new FileSplit(nodeName, path + "_$invlists");
+        }
+        return new Pair<IFileSplitProvider, IFileSplitProvider>(new ConstantFileSplitProvider(btreeSplits),
+                new ConstantFileSplitProvider(invListsSplits));
+    }
+
     private FileSplit[] splitsForInternalOrFeedDataset(String datasetName, String targetIdxName)
-            throws AlgebricksException, MetadataException {
+            throws AlgebricksException {
 
         File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
-        AqlCompiledDatasetDecl adecl = findDataset(datasetName);
-        if (adecl.getDatasetType() != DatasetType.INTERNAL & adecl.getDatasetType() != DatasetType.FEED) {
+        Dataset dataset = findDataset(datasetName);
+        if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
             throw new AlgebricksException("Not an internal or feed dataset");
         }
-        AqlCompiledInternalDatasetDetails compiledDatasetDetails = (AqlCompiledInternalDatasetDetails) adecl
-                .getAqlCompiledDatasetDetails();
-        List<String> nodeGroup = findNodeGroupNodeNames(compiledDatasetDetails.getNodegroupName());
-
+        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+        List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
         if (nodeGroup == null) {
-            throw new AlgebricksException("Couldn't find node group " + compiledDatasetDetails.getNodegroupName());
+            throw new AlgebricksException("Couldn't find node group " + datasetDetails.getNodeGroupName());
         }
 
         List<FileSplit> splitArray = new ArrayList<FileSplit>();
-
         for (String nd : nodeGroup) {
             String[] nodeStores = stores.get(nd);
             if (nodeStores == null) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 4ae38eb..b7ba1a1 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -21,12 +21,13 @@
 import java.util.Set;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
@@ -41,12 +42,11 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 
 public class AqlDataSource implements IDataSource<AqlSourceId> {
 
     private AqlSourceId id;
-    private AqlCompiledDatasetDecl adecl;
+    private Dataset dataset;
     private IAType[] schemaTypes;
     private INodeDomain domain;
     private AqlDataSourceType datasourceType;
@@ -58,14 +58,14 @@
         EXTERNAL_FEED
     }
 
-    public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType datasourceType)
+    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType datasourceType)
             throws AlgebricksException {
         this.id = id;
-        this.adecl = adecl;
+        this.dataset = dataset;
         this.datasourceType = datasourceType;
         switch (datasourceType) {
             case FEED:
-                initFeedDataset(itemType, adecl);
+                initFeedDataset(itemType, dataset);
             case INTERNAL: {
                 initInternalDataset(itemType);
                 break;
@@ -81,12 +81,12 @@
         }
     }
 
-    public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
+    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
         this.id = id;
-        this.adecl = adecl;
-        switch (adecl.getDatasetType()) {
+        this.dataset = dataset;
+        switch (dataset.getDatasetType()) {
             case FEED:
-                initFeedDataset(itemType, adecl);
+                initFeedDataset(itemType, dataset);
                 break;
             case INTERNAL:
                 initInternalDataset(itemType);
@@ -101,32 +101,32 @@
         }
     }
 
+    // TODO: Seems like initFeedDataset() could simply call this method.
     private void initInternalDataset(IAType itemType) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(adecl);
-        int n = partitioningFunctions.size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        ARecordType recordType = (ARecordType) itemType;
+        int n = partitioningKeys.size();
         schemaTypes = new IAType[n + 1];
         for (int i = 0; i < n; i++) {
-            schemaTypes[i] = partitioningFunctions.get(i).third;
+            schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
         }
         schemaTypes[n] = itemType;
-        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
     }
 
-    private void initFeedDataset(IAType itemType, AqlCompiledDatasetDecl decl) {
-
-        if (decl.getAqlCompiledDatasetDetails() instanceof AqlCompiledExternalDatasetDetails) {
+    private void initFeedDataset(IAType itemType, Dataset dataset) {
+        if (dataset.getDatasetDetails() instanceof ExternalDatasetDetails) {
             initExternalDataset(itemType);
         } else {
-            List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                    .getPartitioningFunctions(adecl);
-            int n = partitioningFunctions.size();
+            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            int n = partitioningKeys.size();
             schemaTypes = new IAType[n + 1];
+            ARecordType recordType = (ARecordType) itemType;
             for (int i = 0; i < n; i++) {
-                schemaTypes[i] = partitioningFunctions.get(i).third;
+                schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
             }
             schemaTypes[n] = itemType;
-            domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+            domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
         }
     }
 
@@ -152,8 +152,8 @@
         return id;
     }
 
-    public AqlCompiledDatasetDecl getCompiledDatasetDecl() {
-        return adecl;
+    public Dataset getDataset() {
+        return dataset;
     }
 
     @Override
@@ -170,7 +170,7 @@
 
     @Override
     public IDataSourcePropertiesProvider getPropertiesProvider() {
-        return new AqlDataSourcePartitioningProvider(adecl.getDatasetType(), domain);
+        return new AqlDataSourcePartitioningProvider(dataset.getDatasetType(), domain);
     }
 
     @Override
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
index 4a6d6e4..bac7733 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
@@ -15,24 +15,26 @@
 
 package edu.uci.ics.asterix.metadata.declared;
 
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 
 public class AqlIndex implements IDataSourceIndex<String, AqlSourceId> {
 
-    private final AqlCompiledIndexDecl acid;
+    private final Index index;
     private final AqlCompiledMetadataDeclarations acmd;
     private final String datasetName;
 
     // Every transactions needs to work with its own instance of an
     // AqlMetadataProvider.
-    public AqlIndex(AqlCompiledIndexDecl acid, AqlCompiledMetadataDeclarations acmd, String datasetName) {
-        this.acid = acid;
+    public AqlIndex(Index index, AqlCompiledMetadataDeclarations acmd, String datasetName) {
+        this.index = index;
         this.acmd = acmd;
         this.datasetName = datasetName;
     }
 
+    // TODO: Maybe Index can directly implement IDataSourceIndex<String, AqlSourceId>
     @Override
     public IDataSource<AqlSourceId> getDataSource() {
         try {
@@ -45,7 +47,7 @@
 
     @Override
     public String getId() {
-        return acid.getIndexName();
+        return index.getIndexName();
     }
 
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 3affe24..a3d3472 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -34,7 +34,10 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -47,12 +50,10 @@
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
@@ -61,8 +62,8 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -80,7 +81,6 @@
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
@@ -118,28 +118,27 @@
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
             IDataSource<AqlSourceId> dataSource, List<LogicalVariable> scanVariables,
-            List<LogicalVariable> projectVariables, boolean projectPushed, JobGenContext context,
-            JobSpecification jobSpec) throws AlgebricksException {
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(dataSource.getId().getDatasetName());
-        if (adecl == null) {
+            List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException {
+        Dataset dataset = metadata.findDataset(dataSource.getId().getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + dataSource.getId().getDatasetName());
         }
-        switch (adecl.getDatasetType()) {
+        switch (dataset.getDatasetType()) {
             case FEED:
                 if (dataSource instanceof ExternalFeedDataSource) {
-                    return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+                    return buildExternalDatasetScan(jobSpec, dataset, dataSource);
                 } else {
-                    return buildInternalDatasetScan(jobSpec, adecl, dataSource, context);
+                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
+                            context);
                 }
-
             case INTERNAL: {
-                return buildInternalDatasetScan(jobSpec, adecl, dataSource, context);
+                return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource, context);
             }
-
             case EXTERNAL: {
-                return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+                return buildExternalDatasetScan(jobSpec, dataset, dataSource);
             }
-
             default: {
                 throw new IllegalArgumentException();
             }
@@ -147,44 +146,33 @@
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
-            AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource, JobGenContext context)
-            throws AlgebricksException {
+            List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+            Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context) throws AlgebricksException {
         AqlSourceId asid = dataSource.getId();
+        String dataverseName = asid.getDataverseName();
         String datasetName = asid.getDatasetName();
-        String indexName = DatasetUtils.getPrimaryIndex(acedl).getIndexName();
-
-        try {
-            return buildBtreeRuntime(metadata, context, jobSpec, datasetName, acedl, indexName, null, null, true, true);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataverseName, datasetName);
+        return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, metadata, context, false, datasetName,
+                dataset, primaryIndex.getIndexName(), null, null, true, true);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
-            AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
-        String itemTypeName = acedl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+            Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (dataSource instanceof ExternalFeedDataSource) {
-            AqlCompiledFeedDatasetDetails acfdd = (AqlCompiledFeedDatasetDetails) ((ExternalFeedDataSource) dataSource)
-                    .getCompiledDatasetDecl().getAqlCompiledDatasetDetails();
-
-            return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), acedl.getName(), itemType, acfdd,
-                    metadata.getFormat());
+            FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
+            return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), dataset.getDatasetName(), itemType,
+                    datasetDetails, metadata.getFormat());
         } else {
             return buildExternalDataScannerRuntime(jobSpec, itemType,
-                    (AqlCompiledExternalDatasetDetails) acedl.getAqlCompiledDatasetDetails(), metadata.getFormat());
+                    (ExternalDatasetDetails) dataset.getDatasetDetails(), metadata.getFormat());
         }
     }
 
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataScannerRuntime(
-            JobSpecification jobSpec, IAType itemType, AqlCompiledExternalDatasetDetails decl, IDataFormat format)
+            JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format)
             throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Can only scan datasets of records.");
@@ -192,7 +180,7 @@
 
         IDatasourceReadAdapter adapter;
         try {
-            adapter = (IDatasourceReadAdapter) Class.forName(decl.getAdapter()).newInstance();
+            adapter = (IDatasourceReadAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to load the adapter class " + e);
@@ -205,7 +193,7 @@
         ARecordType rt = (ARecordType) itemType;
 
         try {
-            adapter.configure(decl.getProperties(), itemType);
+            adapter.configure(datasetDetails.getProperties(), itemType);
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -215,7 +203,7 @@
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
-                decl.getAdapter(), decl.getProperties(), rt, scannerDesc);
+                datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, scannerDesc);
         dataScanner.setDatasourceAdapter(adapter);
         AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
@@ -247,13 +235,13 @@
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(
             JobSpecification jobSpec, String dataverse, String dataset, IAType itemType,
-            AqlCompiledFeedDatasetDetails decl, IDataFormat format) throws AlgebricksException {
+            FeedDatasetDetails datasetDetails, IDataFormat format) throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Can only consume records.");
         }
         IDatasourceAdapter adapter;
         try {
-            adapter = (IDatasourceAdapter) Class.forName(decl.getAdapter()).newInstance();
+            adapter = (IDatasourceAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to load the adapter class " + e);
@@ -261,7 +249,7 @@
 
         ARecordType rt = (ARecordType) itemType;
         try {
-            adapter.configure(decl.getProperties(), itemType);
+            adapter.configure(datasetDetails.getProperties(), itemType);
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -271,146 +259,73 @@
         RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(dataverse,
-                dataset), decl.getAdapter(), decl.getProperties(), rt, feedDesc);
+                dataset), datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, feedDesc);
 
         AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
     }
 
-    @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
-            JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, AqlCompiledFeedDatasetDetails decl,
+            JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, FeedDatasetDetails datasetDetails,
             String dataverse, String dataset, List<IFeedMessage> feedMessages) throws AlgebricksException {
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-        try {
-            spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
         FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
                 feedMessages);
-
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, spPc.second);
     }
 
-    @SuppressWarnings("rawtypes")
-    public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(
-            AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
-            String datasetName, AqlCompiledDatasetDecl ddecl, String indexName, int[] lowKeyFields,
-            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) throws AlgebricksException {
-        String itemTypeName = ddecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+    public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
+            List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+            AqlCompiledMetadataDeclarations metadata, JobGenContext context, boolean retainInput, String datasetName,
+            Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
+            boolean highKeyInclusive) throws AlgebricksException {
         boolean isSecondary = true;
-        AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(ddecl);
-
-        if (primIdxDecl != null) {
-            isSecondary = !indexName.equals(primIdxDecl.getIndexName());
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        if (primaryIndex != null) {
+            isSecondary = !indexName.equals(primaryIndex.getIndexName());
         }
-
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
-        ISerializerDeserializer[] recordFields;
-        IBinaryComparatorFactory[] comparatorFactories;
-        ITypeTraits[] typeTraits;
-        int numSecondaryKeys = 0;
-        int i = 0;
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+        int numKeys = numPrimaryKeys;
+        int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
         if (isSecondary) {
-            AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(ddecl, indexName);
-            if (cid == null) {
-                throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
-                        + datasetName);
-            }
-            List<String> secondaryKeyFields = cid.getFieldExprs();
-            numSecondaryKeys = secondaryKeyFields.size();
-            int numKeys = numSecondaryKeys + numPrimaryKeys;
-            recordFields = new ISerializerDeserializer[numKeys];
-            typeTraits = new ITypeTraits[numKeys];
-            // comparatorFactories = new
-            // IBinaryComparatorFactory[numSecondaryKeys];
-            comparatorFactories = new IBinaryComparatorFactory[numKeys];
-            if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                throw new AlgebricksException("Only record types can be indexed.");
-            }
-            ARecordType recType = (ARecordType) itemType;
-            for (i = 0; i < numSecondaryKeys; i++) {
-                Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                        secondaryKeyFields.get(i), recType);
-                IAType keyType = keyTypePair.first;
-                ISerializerDeserializer keySerde = metadata.getFormat().getSerdeProvider()
-                        .getSerializerDeserializer(keyType);
-                recordFields[i] = keySerde;
-                comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                        keyType, true);
-                typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            }
-        } else {
-            recordFields = new ISerializerDeserializer[numPrimaryKeys + 1];
-            comparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
-            typeTraits = new ITypeTraits[numPrimaryKeys + 1];
-            ISerializerDeserializer payloadSerde = metadata.getFormat().getSerdeProvider()
-                    .getSerializerDeserializer(itemType);
-            recordFields[numPrimaryKeys] = payloadSerde;
-            typeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
+            Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+            int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+            numKeys += numSecondaryKeys;
+            keysStartIndex = outputRecDesc.getFieldCount() - numKeys;
         }
-
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(ddecl)) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = metadata.getFormat().getSerdeProvider()
-                    .getSerializerDeserializer(keyType);
-            recordFields[i] = keySerde;
-            // if (!isSecondary) {
-            comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
-                    true);
-            // }
-            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            ++i;
-        }
+        IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                outputVars, keysStartIndex, numKeys, typeEnv, context);
+        ITypeTraits[] typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv,
+                context);
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        RecordDescriptor recDesc = new RecordDescriptor(recordFields);
-
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
         try {
             spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         } catch (Exception e) {
             throw new AlgebricksException(e);
         }
-
-        BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, recDesc,
+        BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
                 appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
                 comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                new BTreeDataflowHelperFactory(), retainInput, NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
     }
 
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(
             AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
-            String datasetName, AqlCompiledDatasetDecl ddecl, String indexName, int[] keyFields)
-            throws AlgebricksException {
-        String itemTypeName = ddecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+            String datasetName, Dataset dataset, String indexName, int[] keyFields) throws AlgebricksException {
+        ARecordType recType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         boolean isSecondary = true;
-        AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(ddecl);
-        if (primIdxDecl != null) {
-            isSecondary = !indexName.equals(primIdxDecl.getIndexName());
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        if (primaryIndex != null) {
+            isSecondary = !indexName.equals(primaryIndex.getIndexName());
         }
 
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         ISerializerDeserializer[] recordFields;
         IBinaryComparatorFactory[] comparatorFactories;
         ITypeTraits[] typeTraits;
@@ -418,82 +333,64 @@
         int numSecondaryKeys = 0;
         int numNestedSecondaryKeyFields = 0;
         int i = 0;
-        if (isSecondary) {
-            AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(ddecl, indexName);
-            if (cid == null) {
-                throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
-                        + datasetName);
-            }
-            List<String> secondaryKeyFields = cid.getFieldExprs();
-            numSecondaryKeys = secondaryKeyFields.size();
-
-            if (numSecondaryKeys != 1) {
-                throw new AlgebricksException(
-                        "Cannot use "
-                                + numSecondaryKeys
-                                + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
-            }
-
-            if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                throw new AlgebricksException("Only record types can be indexed.");
-            }
-            ARecordType recType = (ARecordType) itemType;
-
-            Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                    secondaryKeyFields.get(0), recType);
-            IAType keyType = keyTypePair.first;
-            if (keyType == null) {
-                throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
-            }
-
-            int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
-            numNestedSecondaryKeyFields = dimension * 2;
-
-            int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
-            recordFields = new ISerializerDeserializer[numFields];
-            typeTraits = new ITypeTraits[numFields];
-            comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
-            valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
-
-            IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-            for (i = 0; i < numNestedSecondaryKeyFields; i++) {
-                ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                        .getSerializerDeserializer(nestedKeyType);
-                recordFields[i] = keySerde;
-                comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                        nestedKeyType, true);
-                typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
-                valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
-            }
-        } else {
+        if (!isSecondary) {
             throw new AlgebricksException("R-tree can only be used as a secondary index");
         }
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex == null) {
+            throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
+                    + datasetName);
+        }
+        List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
+        numSecondaryKeys = secondaryKeyFields.size();
+        if (numSecondaryKeys != 1) {
+            throw new AlgebricksException(
+                    "Cannot use "
+                            + numSecondaryKeys
+                            + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
+        }
+        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
+        IAType keyType = keyTypePair.first;
+        if (keyType == null) {
+            throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+        }
+        int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+        numNestedSecondaryKeyFields = dimension * 2;
 
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(ddecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
+        recordFields = new ISerializerDeserializer[numFields];
+        typeTraits = new ITypeTraits[numFields];
+        comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+        valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+
+        IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+        for (i = 0; i < numNestedSecondaryKeyFields; i++) {
             ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(keyType);
+                    .getSerializerDeserializer(nestedKeyType);
             recordFields[i] = keySerde;
-            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+                    nestedKeyType, true);
+            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
+            valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+        }
+
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType type = recType.getFieldType(partitioningKey);
+            ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
+                    .getSerializerDeserializer(type);
+            recordFields[i] = keySerde;
+            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
             ++i;
         }
-
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
         RecordDescriptor recDesc = new RecordDescriptor(recordFields);
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-        try {
-            spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, recDesc,
                 appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
-                comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
+                comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories), false,
                 NoOpOperationCallbackProvider.INSTANCE);
-
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
     }
 
@@ -516,20 +413,19 @@
     public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
             throws AlgebricksException {
         AqlDataSource ads = findDataSource(dataSourceId);
-        AqlCompiledDatasetDecl adecl = ads.getCompiledDatasetDecl();
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        Dataset dataset = ads.getDataset();
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("No index for external dataset " + dataSourceId);
         }
 
-        String idxName = (String) indexId;
-        AqlCompiledIndexDecl acid = DatasetUtils.findSecondaryIndexByName(adecl, idxName);
-        AqlSourceId asid = (AqlSourceId) dataSourceId;
-        if (acid != null) {
-            return new AqlIndex(acid, metadata, asid.getDatasetName());
+        String indexName = (String) indexId;
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex != null) {
+            return new AqlIndex(secondaryIndex, metadata, dataset.getDatasetName());
         } else {
-            AqlCompiledIndexDecl primIdx = DatasetUtils.getPrimaryIndex(adecl);
-            if (primIdx.getIndexName().equals(indexId)) {
-                return new AqlIndex(primIdx, metadata, asid.getDatasetName());
+            Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+            if (primaryIndex.getIndexName().equals(indexId)) {
+                return new AqlIndex(primaryIndex, metadata, dataset.getDatasetName());
             } else {
                 return null;
             }
@@ -541,29 +437,29 @@
         if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
             return null;
         }
-        AqlCompiledDatasetDecl acdd = metadata.findDataset(aqlId.getDatasetName());
-        if (acdd == null) {
+        Dataset dataset = metadata.findDataset(aqlId.getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Datasource with id " + aqlId + " was not found.");
         }
-        String tName = acdd.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(tName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-        return new AqlDataSource(aqlId, acdd, itemType);
+        String tName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(tName);
+        return new AqlDataSource(aqlId, dataset, itemType);
     }
 
     @Override
     public boolean scannerOperatorIsLeaf(IDataSource<AqlSourceId> dataSource) {
         AqlSourceId asid = dataSource.getId();
         String datasetName = asid.getDatasetName();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = null;
+        try {
+            dataset = metadata.findDataset(datasetName);
+        } catch (AlgebricksException e) {
+            throw new IllegalStateException(e);
+        }
+        if (dataset == null) {
             throw new IllegalArgumentException("Unknown dataset " + datasetName);
         }
-        return adecl.getDatasetType() == DatasetType.EXTERNAL;
+        return dataset.getDatasetType() == DatasetType.EXTERNAL;
     }
 
     @Override
@@ -583,27 +479,23 @@
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        String indexName = primaryIndex.getIndexName();
 
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        String itemTypeName = dataset.getItemTypeName();
+        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
 
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, context.getBinaryComparatorFactoryProvider());
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
@@ -612,16 +504,14 @@
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
-    @Override
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
         String datasetName = dataSource.getId().getDatasetName();
         int numKeys = keys.size();
-        // move key fields to front
+        // Move key fields to front.
         int[] fieldPermutation = new int[numKeys + 1];
-        // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
         int i = 0;
         for (LogicalVariable varKey : keys) {
             int idx = propagatedSchema.findVariable(varKey);
@@ -630,78 +520,75 @@
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        String indexName = primaryIndex.getIndexName();
 
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        String itemTypeName = dataset.getItemTypeName();
+        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
+
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, context.getBinaryComparatorFactoryProvider());
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.INSERT,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
                 new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
     @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
+            LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+            throws AlgebricksException {
+        return getInsertOrDeleteRuntime(IndexOp.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
+                context, spec);
+    }
+
+    @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
-        String datasetName = dataSource.getId().getDatasetName();
-        int numKeys = keys.size();
-        // move key fields to front
-        int[] fieldPermutation = new int[numKeys + 1];
-        // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
-        int i = 0;
-        for (LogicalVariable varKey : keys) {
-            int idx = propagatedSchema.findVariable(varKey);
-            fieldPermutation[i] = idx;
-            i++;
-        }
-        fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
+        return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
+                context, spec);
+    }
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
+            IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
+            IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+            JobGenContext context, JobSpecification spec) throws AlgebricksException {
+        String indexName = dataSourceIndex.getId();
+        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
-
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
-
-        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+        switch (secondaryIndex.getIndexType()) {
+            case BTREE: {
+                return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+                        filterFactory, recordDesc, context, spec, indexOp);
+            }
+            case RTREE: {
+                return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+                        filterFactory, recordDesc, context, spec, indexOp);
+            }
+            default: {
+                throw new AlgebricksException("Insert and delete not implemented for index type: "
+                        + secondaryIndex.getIndexType());
+            }
         }
-
-        TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
     @Override
@@ -710,21 +597,8 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        String indexName = dataSourceIndex.getId();
-        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
-        if (cid.getKind() == IndexKind.BTREE) {
-            return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.INSERT);
-        } else {
-            return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.INSERT);
-        }
+        return getIndexInsertOrDeleteRuntime(IndexOp.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     @Override
@@ -733,21 +607,8 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        String indexName = dataSourceIndex.getId();
-        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
-        if (cid.getKind() == IndexKind.BTREE) {
-            return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.DELETE);
-        } else {
-            return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.DELETE);
-        }
+        return getIndexInsertOrDeleteRuntime(IndexOp.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -760,7 +621,7 @@
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
         IScalarEvaluatorFactory filterEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(filterExpr,
                 typeEnv, inputSchemas, context);
-        return new AsterixTupleFilterFactory(filterEvalFactory, context.getBinaryBooleanInspector());
+        return new AsterixTupleFilterFactory(filterEvalFactory, context.getBinaryBooleanInspectorFactory());
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String datasetName,
@@ -782,37 +643,33 @@
             i++;
         }
 
-        // dataset
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String itemTypeName = compiledDatasetDecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
 
-        // index parameters
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        List<String> secondaryKeyExprs = cid.getFieldExprs();
+        // Index parameters.
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
         ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
         for (i = 0; i < secondaryKeys.size(); ++i) {
-            IAType keyType = AqlCompiledIndexDecl.keyFieldType(secondaryKeyExprs.get(i).toString(), recType);
+            Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(i).toString(),
+                    recType);
+            IAType keyType = keyPairType.first;
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType keyType = recType.getFieldType(partitioningKey);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -820,13 +677,8 @@
         }
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -838,22 +690,16 @@
             String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        String itemTypeName = compiledDatasetDecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Dataset dataset = metadata.findDataset(datasetName);
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        List<String> secondaryKeyExprs = cid.getFieldExprs();
-        Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyExprs.get(0),
-                recType);
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(0), recType);
         IAType spatialType = keyPairType.first;
         int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         int numSecondaryKeys = dimension * 2;
@@ -877,16 +723,14 @@
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
         IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
         for (i = 0; i < numSecondaryKeys; i++) {
-            ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(nestedKeyType);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
                     nestedKeyType, true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
             valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
         }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType keyType = recType.getFieldType(partitioningKey);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -894,13 +738,8 @@
         }
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -917,10 +756,6 @@
         return new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
     }
 
-    public static ITreeIndexFrameFactory createBTreeNSMLeafFrameFactory(ITypeTraits[] typeTraits) {
-        return new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
-    }
-
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return AsterixBuiltinFunctions.lookupFunction(fid);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
index 16e8bae..566265f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
@@ -1,17 +1,17 @@
 package edu.uci.ics.asterix.metadata.declared;
 
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class ExternalFeedDataSource extends AqlDataSource {
 
-    public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
-        super(id,adecl, itemType);
+    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
+        super(id, dataset, itemType);
     }
 
-    public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType dataSourceType) throws AlgebricksException {
-        super(id,adecl, itemType, dataSourceType);
+    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType dataSourceType)
+            throws AlgebricksException {
+        super(id, dataset, itemType, dataSourceType);
     }
-    
-  
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
deleted file mode 100644
index 2253db9..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public interface IAqlCompiledDatasetDetails {
-
-    public DatasetType getDatasetType();
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index e810bc9..d383955 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -30,16 +30,16 @@
     private final String dataverseName;
     // Enforced to be unique within a dataverse.
     private final String datasetName;
-    // Type of values stored in this dataset.
-    private final String datatypeName;
+    // Type of items stored in this dataset.
+    private final String itemTypeName;
     private final DatasetType datasetType;
     private IDatasetDetails datasetDetails;
 
-    public Dataset(String dataverseName, String datasetName, String datatypeName, IDatasetDetails datasetDetails,
+    public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
             DatasetType datasetType) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
-        this.datatypeName = datatypeName;
+        this.itemTypeName = itemTypeName;
         this.datasetType = datasetType;
         this.datasetDetails = datasetDetails;
     }
@@ -52,11 +52,11 @@
         return datasetName;
     }
 
-    public String getDatatypeName() {
-        return datatypeName;
+    public String getItemTypeName() {
+        return itemTypeName;
     }
 
-    public DatasetType getType() {
+    public DatasetType getDatasetType() {
         return datasetType;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
index 36db14a..9761c47 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
@@ -92,7 +92,7 @@
 
         // write field 2
         listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[2]);
-        for (String field : partitioningKey) {
+        for (String field : partitioningKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -104,7 +104,7 @@
 
         // write field 3
         listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[3]);
-        for (String field : primaryKey) {
+        for (String field : primaryKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
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 4a648fe..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
@@ -19,6 +19,12 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
 /**
  * Metadata describing an index.
@@ -35,6 +41,19 @@
     private final IndexType indexType;
     private final List<String> keyFieldNames;
     private final boolean isPrimaryIndex;
+    // Specific to NGRAM indexes.
+    private final int gramLength;
+
+    public Index(String dataverseName, String datasetName, String indexName, IndexType indexType,
+            List<String> keyFieldNames, int gramLength, boolean isPrimaryIndex) {
+        this.dataverseName = dataverseName;
+        this.datasetName = datasetName;
+        this.indexName = indexName;
+        this.indexType = indexType;
+        this.keyFieldNames = keyFieldNames;
+        this.gramLength = gramLength;
+        this.isPrimaryIndex = isPrimaryIndex;
+    }
 
     public Index(String dataverseName, String datasetName, String indexName, IndexType indexType,
             List<String> keyFieldNames, boolean isPrimaryIndex) {
@@ -43,6 +62,7 @@
         this.indexName = indexName;
         this.indexType = indexType;
         this.keyFieldNames = keyFieldNames;
+        this.gramLength = -1;
         this.isPrimaryIndex = isPrimaryIndex;
     }
 
@@ -62,6 +82,10 @@
         return keyFieldNames;
     }
 
+    public int getGramLength() {
+        return gramLength;
+    }
+
     public IndexType getIndexType() {
         return indexType;
     }
@@ -73,4 +97,30 @@
     public boolean isSecondaryIndex() {
         return !isPrimaryIndex();
     }
+
+    public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
+            throws AlgebricksException {
+        IAType keyType = Index.keyFieldType(expr, recType);
+        boolean nullable = false;
+        if (keyType.getTypeTag() == ATypeTag.UNION) {
+            AUnionType unionType = (AUnionType) keyType;
+            if (unionType.isNullableType()) {
+                // The non-null type is always at index 1.
+                keyType = unionType.getUnionList().get(1);
+                nullable = true;
+            }
+        }
+        return new Pair<IAType, Boolean>(keyType, nullable);
+    }
+
+    private static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
+        String[] names = recType.getFieldNames();
+        int n = names.length;
+        for (int i = 0; i < n; i++) {
+            if (names[i].equals(expr)) {
+                return recType.getFieldTypes()[i];
+            }
+        }
+        throw new AlgebricksException("Could not find field " + expr + " in the schema.");
+    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
index de51dc8..53d10e4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
@@ -48,29 +48,29 @@
 
     protected final FileStructure fileStructure;
     protected final PartitioningStrategy partitioningStrategy;
-    protected final List<String> partitioningKey;
-    protected final List<String> primaryKey;
-    protected final String groupName;
+    protected final List<String> partitioningKeys;
+    protected final List<String> primaryKeys;
+    protected final String nodeGroupName;
 
     public InternalDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
             List<String> partitioningKey, List<String> primaryKey, String groupName) {
         this.fileStructure = fileStructure;
         this.partitioningStrategy = partitioningStrategy;
-        this.partitioningKey = partitioningKey;
-        this.primaryKey = primaryKey;
-        this.groupName = groupName;
+        this.partitioningKeys = partitioningKey;
+        this.primaryKeys = primaryKey;
+        this.nodeGroupName = groupName;
     }
 
     public String getNodeGroupName() {
-        return this.groupName;
+        return nodeGroupName;
     }
 
     public List<String> getPartitioningKey() {
-        return this.partitioningKey;
+        return partitioningKeys;
     }
 
     public List<String> getPrimaryKey() {
-        return primaryKey;
+        return primaryKeys;
     }
 
     public FileStructure getFileStructure() {
@@ -115,7 +115,7 @@
         // write field 2
         listBuilder
                 .reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX]);
-        for (String field : partitioningKey) {
+        for (String field : partitioningKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -129,7 +129,7 @@
         // write field 3
         listBuilder
                 .reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX]);
-        for (String field : primaryKey) {
+        for (String field : primaryKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
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 78703f3..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;
@@ -209,13 +209,13 @@
 
         // write field 2
         fieldValue.reset();
-        aString.setValue(dataset.getDatatypeName());
+        aString.setValue(dataset.getItemTypeName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATATYPENAME_FIELD_INDEX, fieldValue);
 
         // write field 3
         fieldValue.reset();
-        aString.setValue(dataset.getType().toString());
+        aString.setValue(dataset.getDatasetType().toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETTYPE_FIELD_INDEX, fieldValue);
 
@@ -241,7 +241,7 @@
             throws HyracksDataException {
 
         dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
-        switch (dataset.getType()) {
+        switch (dataset.getDatasetType()) {
             case INTERNAL:
                 recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX, fieldValue);
                 break;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 34d4162..3c8c48a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -31,11 +31,13 @@
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AOrderedList;
 import edu.uci.ics.asterix.om.base.ARecord;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.base.IACursor;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -53,11 +55,17 @@
     public static final int INDEX_INDEXNAME_TUPLE_FIELD_INDEX = 2;
     // Payload field containing serialized Index.
     public static final int INDEX_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+    // Field name of open field.
+    public static final String GRAM_LENGTH_FIELD_NAME = "GramLength";
 
     private IAOrderedListBuilder listBuilder = new OrderedListBuilder();
+    private ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
     private ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
     private List<String> searchKey;
     @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.INDEX_RECORDTYPE);
 
@@ -89,7 +97,13 @@
         }
         Boolean isPrimaryIndex = ((ABoolean) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
                 .getBoolean();
-        return new Index(dvName, dsName, indexName, indexStructure, searchKey, isPrimaryIndex);
+        // Check if there is a gram length as well.
+        int gramLength = -1;
+        int gramLenPos = rec.getType().findFieldPosition(GRAM_LENGTH_FIELD_NAME);
+        if (gramLenPos >= 0) {
+            gramLength = ((AInt32) rec.getValueByPos(gramLenPos)).getIntegerValue();
+        }
+        return new Index(dvName, dsName, indexName, indexStructure, searchKey, gramLength, isPrimaryIndex);
     }
 
     @Override
@@ -161,6 +175,16 @@
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
 
+        // write optional field 7        
+        if (instance.getGramLength() > 0) {
+            fieldValue.reset();
+            nameValue.reset();
+            aString.setValue(GRAM_LENGTH_FIELD_NAME);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            intSerde.serialize(new AInt32(instance.getGramLength()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+        }
+
         // write record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index dd05b00..edb3808 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -1,8 +1,8 @@
 package edu.uci.ics.asterix.metadata.functions;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -53,11 +53,11 @@
                 AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
                 String datasetName = ((AString) acv.getObject()).getStringValue();
                 AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
-                String tn = acdd.getItemTypeName();
+                String tn = dataset.getItemTypeName();
                 IAType t2 = metadata.findType(tn);
                 if (t2 == null) {
                     throw new AlgebricksException("No type for dataset " + datasetName);
@@ -89,11 +89,11 @@
                 AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
                 String datasetName = ((AString) acv.getObject()).getStringValue();
                 AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
-                String tn = acdd.getItemTypeName();
+                String tn = dataset.getItemTypeName();
                 IAType t2 = metadata.findType(tn);
                 if (t2 == null) {
                     throw new AlgebricksException("No type for dataset " + datasetName);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
index a34f14b..cead5f2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
@@ -4,115 +4,78 @@
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class DatasetUtils {
-    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(
-            AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
+            ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
             throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[numKeys];
-        for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+        }
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
         }
         return bcfs;
     }
 
-    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(
-            AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryHashFunctionFactoryProvider hashFunProvider)
-            throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
+            IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
-                .getPartitioningFunctions();
-        int numKeys = partitioningFunctions.size();
-        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[numKeys];
-        for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+        }
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             bhffs[i] = hashFunProvider.getBinaryHashFunctionFactory(keyType);
         }
         return bhffs;
     }
 
-    public static ITypeTraits[] computeTupleTypeTraits(AqlCompiledDatasetDecl compiledDatasetDecl,
-            AqlCompiledMetadataDeclarations datasetDecls) throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+    public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
-                .getPartitioningFunctions();
-        int numKeys = partitioningFunctions.size();
+        }
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
         ITypeTraits[] typeTraits = new ITypeTraits[numKeys + 1];
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
-        IAType payloadType = datasetDecls.findType(compiledDatasetDecl.getItemTypeName());
-        typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(payloadType);
+        typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
         return typeTraits;
     }
 
-    public static List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions(
-            AqlCompiledDatasetDecl decl) {
-        return ((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()).getPartitioningFunctions();
+    public static List<String> getPartitioningKeys(Dataset dataset) {
+        return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
     }
 
-    public static String getNodegroupName(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getNodegroupName();
+    public static String getNodegroupName(Dataset dataset) {
+        return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
     }
 
-    public static AqlCompiledIndexDecl getPrimaryIndex(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPrimaryIndex();
-
+    public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            if (partitioningKeys.get(i).equals(fieldExpr)) {
+                return i;
+            }
+        }
+        return -1;
     }
-
-    public static AqlCompiledIndexDecl findSecondaryIndexByName(AqlCompiledDatasetDecl decl, String indexName) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())
-                .findSecondaryIndexByName(indexName));
-    }
-
-    public static List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(AqlCompiledDatasetDecl decl,
-            String fieldExpr) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
-                .findSecondaryIndexesByOneOfTheKeys(fieldExpr);
-    }
-
-    public static int getPositionOfPartitioningKeyField(AqlCompiledDatasetDecl decl, String fieldExpr) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
-                .getPositionOfPartitioningKeyField(fieldExpr);
-    }
-
-    public static List<String> getPartitioningExpressions(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPartitioningExprs();
-    }
-
-    public static List<AqlCompiledIndexDecl> getSecondaryIndexes(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getSecondaryIndexes();
-    }
-
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
new file mode 100644
index 0000000..cb6838e
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
@@ -0,0 +1,57 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+
+public class AListElementToken implements IToken {
+
+    protected byte[] data;
+    protected int start;
+    protected int length;
+    protected int tokenLength;
+    protected int typeTag;
+    
+    @Override
+    public byte[] getData() {
+        return data;
+    }
+
+    @Override
+    public int getLength() {
+        return length;
+    }
+
+    @Override
+    public int getStart() {
+        return start;
+    }
+
+    @Override
+    public int getTokenLength() {
+        return tokenLength;
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length, int tokenLength, int tokenCount) {
+        this.data = data;
+        this.start = start;
+        this.length = length;
+        this.tokenLength = tokenLength;
+        // We abuse the last param, tokenCount, to pass the type tag.
+        typeTag = tokenCount;
+    }
+
+    @Override
+    public void serializeToken(DataOutput dos) throws IOException {
+        dos.writeByte(typeTag);
+        dos.write(data, start, length);
+    }
+
+    @Override
+    public void serializeTokenCount(DataOutput dos) throws IOException {
+        throw new UnsupportedOperationException("Token count not implemented.");
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
new file mode 100644
index 0000000..fc46039
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+
+public class AListElementTokenFactory implements ITokenFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IToken createToken() {
+        return new AListElementToken();
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
new file mode 100644
index 0000000..667bfe7
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -0,0 +1,69 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+
+public class AOrderedListBinaryTokenizer implements IBinaryTokenizer {
+
+    protected byte[] data;
+    protected int start;
+    protected int length;
+    protected int listLength;
+    protected int itemIndex;
+    
+    protected final IToken token;
+    
+    public AOrderedListBinaryTokenizer(ITokenFactory tokenFactory) {
+        token = tokenFactory.createToken();
+    }
+    
+    @Override
+    public IToken getToken() {
+        return token;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return itemIndex < listLength;
+    }
+
+    @Override
+    public void next() {
+        int itemOffset = -1;
+        int length = -1;
+        try {
+            itemOffset = getItemOffset(data, start, itemIndex);
+            // Assuming homogeneous list.
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[start + 1]);
+            length = NonTaggedFormatUtil.getFieldValueLength(data, itemOffset, typeTag, false);
+            // Last param is a hack to pass the type tag.
+            token.reset(data, itemOffset, length, length, data[start + 1]);
+        } catch (AsterixException e) {
+            throw new IllegalStateException(e);
+        }
+        itemIndex++;
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length) {
+        this.data = data;
+        this.start = start;
+        this.length = length;
+        this.listLength = getNumberOfItems(data, start);
+        this.itemIndex = 0;
+    }
+    
+    protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
+        return AOrderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
+    }
+    
+    protected int getNumberOfItems(byte[] data, int start) {
+        return AOrderedListSerializerDeserializer.getNumberOfItems(data, start);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
new file mode 100644
index 0000000..d7712ea
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+
+public class AOrderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ITokenFactory tokenFactory;
+    
+    public AOrderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
+        this.tokenFactory = tokenFactory;
+    }
+    
+    @Override
+    public IBinaryTokenizer createTokenizer() {
+        return new AOrderedListBinaryTokenizer(tokenFactory);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
new file mode 100644
index 0000000..11ab251
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+
+public class AUnorderedListBinaryTokenizer extends AOrderedListBinaryTokenizer {
+
+    public AUnorderedListBinaryTokenizer(ITokenFactory tokenFactory) {
+        super(tokenFactory);
+    }
+    
+    @Override
+    protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
+        return AUnorderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
+    }
+
+    @Override
+    protected int getNumberOfItems(byte[] data, int start) {
+        return AUnorderedListSerializerDeserializer.getNumberOfItems(data, start);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
new file mode 100644
index 0000000..54b7692
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+
+public class AUnorderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
+    
+    private static final long serialVersionUID = 1L;
+    private final ITokenFactory tokenFactory;
+    
+    public AUnorderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
+        this.tokenFactory = tokenFactory;
+    }
+    
+    @Override
+    public IBinaryTokenizer createTokenizer() {
+        return new AUnorderedListBinaryTokenizer(tokenFactory);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
index a8e9cb9..cddb014 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
@@ -1,7 +1,9 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
+import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public interface IBinaryTokenizerFactoryProvider {
-    public IBinaryTokenizerFactory getTokenizerFactory(Object type);
+    public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens);
+    public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost, boolean hashedTokens);
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
new file mode 100644
index 0000000..2a7fdb5
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import java.util.Collections;
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
+
+// TODO: Should go into hyracks.
+public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
+
+    private int edThresh;
+
+    public ListEditDistanceSearchModifier(int edThresh) {
+        this.edThresh = edThresh;
+    }
+
+    @Override
+    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
+        return invListCursors.size() - edThresh;
+    }
+
+    @Override
+    public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
+        Collections.sort(invListCursors);
+        return invListCursors.size() - getOccurrenceThreshold(invListCursors) + 1;
+    }
+
+    public int getEdThresh() {
+        return edThresh;
+    }
+
+    public void setEdThresh(int edThresh) {
+        this.edThresh = edThresh;
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
new file mode 100644
index 0000000..b117a36
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.dataflow.data.common;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+// TODO: Should go into hyracks.
+public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int edThresh;
+    
+    public ListEditDistanceSearchModifierFactory(int edThresh) {
+        this.edThresh = edThresh;
+    }
+    
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new ListEditDistanceSearchModifier(edThresh);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index 3653d46..7299b7c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -120,7 +120,7 @@
 
     public static int getItemOffset(byte[] serOrderedList, int offset, int itemIndex) throws AsterixException {
         if (serOrderedList[offset] == ATypeTag.ORDEREDLIST.serialize()) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[1]);
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + 1]);
             switch (typeTag) {
                 case STRING:
                 case RECORD:
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 694d189..3b87368 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -138,10 +138,11 @@
                     openFields[i] = AObjectSerializerDeserializer.INSTANCE.deserialize(in);
                     fieldTypes[i] = openFields[i].getType();
                 }
-                this.recordType = new ARecordType(null, fieldNames, fieldTypes, true);
+                ARecordType openPartRecType = new ARecordType(null, fieldNames, fieldTypes, true);
                 if (numberOfSchemaFields > 0) {
-                    ARecordType mergedRecordType = mergeRecordTypes(this.recordType, recordType);
-                    return new ARecord(mergedRecordType, mergeFields(closedFields, openFields));
+                    ARecordType mergedRecordType = mergeRecordTypes(this.recordType, openPartRecType);
+                    IAObject[] mergedFields = mergeFields(closedFields, openFields);
+                    return new ARecord(mergedRecordType, mergedFields);
                 } else {
                     return new ARecord(this.recordType, openFields);
                 }
@@ -154,16 +155,14 @@
     }
 
     private IAObject[] mergeFields(IAObject[] closedFields, IAObject[] openFields) {
-
         IAObject[] fields = new IAObject[closedFields.length + openFields.length];
-
         int i = 0;
-        for (; i < closedFields.length; i++)
+        for (; i < closedFields.length; i++) {
             fields[i] = closedFields[i];
-
-        for (int j = 0; j < openFields.length; j++)
-            fields[i] = closedFields[j];
-
+        }
+        for (int j = 0; j < openFields.length; j++) {
+            fields[closedFields.length + j] = openFields[j];
+        }
         return fields;
     }
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index 1eadeb2..f909b46 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -121,14 +121,14 @@
 
     public static int getItemOffset(byte[] serOrderedList, int offset, int itemIndex) throws AsterixException {
         if (serOrderedList[offset] == ATypeTag.UNORDEREDLIST.serialize()) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[1]);
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + 1]);
             switch (typeTag) {
                 case STRING:
                 case RECORD:
                 case ORDEREDLIST:
                 case UNORDEREDLIST:
                 case ANY:
-                    return AInt32SerializerDeserializer.getInt(serOrderedList, offset + 10 + (4 * itemIndex));
+                    return offset + AInt32SerializerDeserializer.getInt(serOrderedList, offset + 10 + (4 * itemIndex));
                 default:
                     int length = NonTaggedFormatUtil.getFieldValueLength(serOrderedList, offset + 1, typeTag, true);
                     return offset + 10 + (length * itemIndex);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
index 9da5b8b..c1df096 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.formats.base;
 
-
 import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -12,10 +11,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
@@ -35,9 +34,9 @@
 
     public ITypeTraitProvider getTypeTraitProvider();
 
-    public IBinaryBooleanInspector getBinaryBooleanInspector();
+    public IBinaryBooleanInspectorFactory getBinaryBooleanInspectorFactory();
 
-    public IBinaryIntegerInspector getBinaryIntegerInspector();
+    public IBinaryIntegerInspectorFactory getBinaryIntegerInspectorFactory();
 
     public IPrinterFactoryProvider getPrinterFactoryProvider();
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
index b9dfe9a..97a0c13 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
@@ -2,12 +2,18 @@
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 public class AqlBinaryBooleanInspectorImpl implements IBinaryBooleanInspector {
+    public static final IBinaryBooleanInspectorFactory FACTORY = new IBinaryBooleanInspectorFactory() {
+        private static final long serialVersionUID = 1L;
 
-    private static final long serialVersionUID = 1L;
-
-    public static final AqlBinaryBooleanInspectorImpl INSTANCE = new AqlBinaryBooleanInspectorImpl();
+        @Override
+        public IBinaryBooleanInspector createBinaryBooleanInspector(IHyracksTaskContext ctx) {
+            return new AqlBinaryBooleanInspectorImpl();
+        }
+    };
 
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
index f972a1c..ede7b99 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
@@ -7,6 +7,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.BooleanBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.LongBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.RectangleBinaryComparatorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -22,15 +23,34 @@
 
     private static final long serialVersionUID = 1L;
     public static final AqlBinaryComparatorFactoryProvider INSTANCE = new AqlBinaryComparatorFactoryProvider();
-    public static final PointableBinaryComparatorFactory INTEGER_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(IntegerPointable.FACTORY);
-    public static final PointableBinaryComparatorFactory FLOAT_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(FloatPointable.FACTORY);
-    public static final PointableBinaryComparatorFactory DOUBLE_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(DoublePointable.FACTORY);
-    public static final PointableBinaryComparatorFactory UTF8STRING_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY);
-    
-    
+    public static final PointableBinaryComparatorFactory INTEGER_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(
+            IntegerPointable.FACTORY);
+    public static final PointableBinaryComparatorFactory FLOAT_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(
+            FloatPointable.FACTORY);
+    public static final PointableBinaryComparatorFactory DOUBLE_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(
+            DoublePointable.FACTORY);
+    public static final PointableBinaryComparatorFactory UTF8STRING_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(
+            UTF8StringPointable.FACTORY);
+    // Equivalent to UTF8STRING_POINTABLE_INSTANCE but all characters are considered lower case to implement case-insensitive comparisons.    
+    public static final PointableBinaryComparatorFactory UTF8STRING_LOWERCASE_POINTABLE_INSTANCE = new PointableBinaryComparatorFactory(
+            UTF8StringLowercasePointable.FACTORY);
+
     private AqlBinaryComparatorFactoryProvider() {
     }
 
+    // This method add the option of ignoring the case in string comparisons.
+    // TODO: We should incorporate this option more nicely, but I'd have to change algebricks.
+    public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending, boolean ignoreCase) {
+        if (type == null) {
+            return anyBinaryComparatorFactory(ascending);
+        }
+        IAType aqlType = (IAType) type;
+        if (aqlType.getTypeTag() == ATypeTag.STRING && ignoreCase) {
+            return addOffset(UTF8STRING_LOWERCASE_POINTABLE_INSTANCE, ascending);
+        }
+        return getBinaryComparatorFactory(type, ascending);
+    }
+
     @Override
     public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending) {
         if (type == null) {
@@ -119,7 +139,7 @@
         if (ascending) {
             return AObjectAscBinaryComparatorFactory.INSTANCE;
         } else {
-            return AObjectDescBinaryComparatorFactory.INSTANCE;            
+            return AObjectDescBinaryComparatorFactory.INSTANCE;
         }
     }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
index f2dd95d..cab8ded 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
@@ -27,7 +27,8 @@
     public static final PointableBinaryHashFunctionFactory FLOAT_POINTABLE_INSTANCE = new PointableBinaryHashFunctionFactory(FloatPointable.FACTORY);
     public static final PointableBinaryHashFunctionFactory DOUBLE_POINTABLE_INSTANCE = new PointableBinaryHashFunctionFactory(DoublePointable.FACTORY);
     public static final PointableBinaryHashFunctionFactory UTF8STRING_POINTABLE_INSTANCE = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY);
- 
+    // Equivalent to UTF8STRING_POINTABLE_INSTANCE but all characters are considered lower case to implement case-insensitive hashing.    
+    public static final PointableBinaryHashFunctionFactory UTF8STRING_LOWERCASE_POINTABLE_INSTANCE = new PointableBinaryHashFunctionFactory(UTF8StringLowercasePointable.FACTORY);
     
     private AqlBinaryHashFunctionFactoryProvider() {
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryIntegerInspector.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
index d63d3c5..d7fa67e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryIntegerInspector.java
@@ -1,12 +1,19 @@
 package edu.uci.ics.asterix.formats.nontagged;
 
 import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 
 public class AqlBinaryIntegerInspector implements IBinaryIntegerInspector {
+    public static final IBinaryIntegerInspectorFactory FACTORY = new IBinaryIntegerInspectorFactory() {
+        private static final long serialVersionUID = 1L;
 
-    private static final long serialVersionUID = 1L;
-    public static final AqlBinaryIntegerInspector INSTANCE = new AqlBinaryIntegerInspector();
+        @Override
+        public IBinaryIntegerInspector createBinaryIntegerInspector(IHyracksTaskContext ctx) {
+            return new AqlBinaryIntegerInspector();
+        }
+    };
 
     private AqlBinaryIntegerInspector() {
     }
@@ -15,5 +22,4 @@
     public int getIntegerValue(byte[] bytes, int offset, int length) {
         return IntegerSerializerDeserializer.getInt(bytes, offset + 1);
     }
-
-}
+}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
index 9893273..859b0f2 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
@@ -1,9 +1,14 @@
 package edu.uci.ics.asterix.formats.nontagged;
 
 import edu.uci.ics.asterix.dataflow.data.common.IBinaryTokenizerFactoryProvider;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.dataflow.data.common.AListElementTokenFactory;
+import edu.uci.ics.asterix.dataflow.data.common.AOrderedListBinaryTokenizerFactory;
+import edu.uci.ics.asterix.dataflow.data.common.AUnorderedListBinaryTokenizerFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
 
 public class AqlBinaryTokenizerFactoryProvider implements IBinaryTokenizerFactoryProvider {
@@ -11,20 +16,58 @@
     public static final AqlBinaryTokenizerFactoryProvider INSTANCE = new AqlBinaryTokenizerFactoryProvider();
 
     private static final IBinaryTokenizerFactory aqlStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
-            true, true, new UTF8WordTokenFactory());
+            true, true, new UTF8WordTokenFactory(ATypeTag.STRING.serialize(), ATypeTag.INT32.serialize()));
+    
+    private static final IBinaryTokenizerFactory aqlHashingStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
+            true, true, new HashedUTF8WordTokenFactory(ATypeTag.INT32.serialize(), ATypeTag.INT32.serialize()));
 
+    private static final IBinaryTokenizerFactory orderedListTokenizer = new AOrderedListBinaryTokenizerFactory(new AListElementTokenFactory());
+    
+    private static final IBinaryTokenizerFactory unorderedListTokenizer = new AUnorderedListBinaryTokenizerFactory(new AListElementTokenFactory());
+    
     @Override
-    public IBinaryTokenizerFactory getTokenizerFactory(Object type) {
-        IAType aqlType = (IAType) type;
-        switch (aqlType.getTypeTag()) {
+    public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens) {
+        switch (typeTag) {
             case STRING: {
-                return aqlStringTokenizer;
+                if (hashedTokens) {
+                    return aqlHashingStringTokenizer;
+                } else {
+                    return aqlStringTokenizer;
+                }
             }
-
+            case ORDEREDLIST: {
+                return orderedListTokenizer;
+            }
+            case UNORDEREDLIST: {
+                return unorderedListTokenizer;
+            }
             default: {
                 return null;
             }
         }
     }
 
+    @Override
+    public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost,
+            boolean hashedTokens) {
+        switch (typeTag) {
+            case STRING: {
+                if (hashedTokens) {
+                    return null;
+                } else {
+                    return new NGramUTF8StringBinaryTokenizerFactory(gramLength, usePrePost, true, true,
+                            new UTF8NGramTokenFactory(ATypeTag.STRING.serialize(), ATypeTag.INT32.serialize()));
+                }
+            }
+            case ORDEREDLIST: {
+                return orderedListTokenizer;
+            }
+            case UNORDEREDLIST: {
+                return unorderedListTokenizer;
+            }
+            default: {
+                return null;
+            }
+        }
+    }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlTypeTraitProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlTypeTraitProvider.java
index 68dbaee..eac6602 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlTypeTraitProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlTypeTraitProvider.java
@@ -17,7 +17,7 @@
     private static final ITypeTraits THIRTYTWOBYTETYPETRAIT = new TypeTrait(32 + 1);
     private static final ITypeTraits TWENTYFOURBYTETYPETRAIT = new TypeTrait(24 + 1);
 
-    private static final ITypeTraits VARLENTYPETRAIT = new TypeTrait(false,-1);
+    private static final ITypeTraits VARLENTYPETRAIT = new TypeTrait(false, -1);
 
     public static final AqlTypeTraitProvider INSTANCE = new AqlTypeTraitProvider();
 
@@ -53,8 +53,6 @@
     }
 }
 
-
-
 class TypeTrait implements ITypeTraits {
 
     @Override
@@ -66,18 +64,17 @@
     public int getFixedLength() {
         return fixedLength;
     }
-   
+
     private boolean isFixedLength;
     private int fixedLength;
-    
-    public TypeTrait(boolean isFixedLength, int fixedLength){
+
+    public TypeTrait(boolean isFixedLength, int fixedLength) {
         this.isFixedLength = isFixedLength;
         this.fixedLength = fixedLength;
     }
-    
-    public TypeTrait(int fixedLength){
+
+    public TypeTrait(int fixedLength) {
         this.isFixedLength = true;
         this.fixedLength = fixedLength;
     }
 }
-
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
new file mode 100644
index 0000000..895dac4
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
@@ -0,0 +1,52 @@
+/**
+ * Copyright 2010-2011 The Regents of the University of California
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.asterix.formats.nontagged;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+
+// TODO: Should be moved into Hyracks with the rest of the tokenizer code.
+public class NGramUTF8StringBinaryTokenizerFactory implements
+		IBinaryTokenizerFactory {
+
+	private static final long serialVersionUID = 1L;
+	private final int gramLength;
+	private final boolean usePrePost;
+	private final boolean ignoreTokenCount;
+	private final boolean sourceHasTypeTag;
+	private final ITokenFactory tokenFactory;
+
+	public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
+			boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+		this.gramLength = gramLength;
+		this.usePrePost = usePrePost;
+	    this.ignoreTokenCount = ignoreTokenCount;
+		this.sourceHasTypeTag = sourceHasTypeTag;
+		this.tokenFactory = tokenFactory;
+	}
+
+    @Override
+    public IBinaryTokenizer createTokenizer() {
+        return new NGramUTF8StringBinaryTokenizer(gramLength, usePrePost, ignoreTokenCount, sourceHasTypeTag,
+                tokenFactory);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/UTF8StringLowercasePointable.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/UTF8StringLowercasePointable.java
new file mode 100644
index 0000000..36c19f00
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/UTF8StringLowercasePointable.java
@@ -0,0 +1,88 @@
+package edu.uci.ics.asterix.formats.nontagged;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+// TODO: Perhaps this class should go into hyracks.
+public final class UTF8StringLowercasePointable extends AbstractPointable implements IHashable, IComparable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new UTF8StringLowercasePointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static int getUTFLen(byte[] b, int s) {
+        return ((b[s] & 0xff) << 8) + ((b[s + 1] & 0xff) << 0);
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        int utflen1 = getUTFLen(this.bytes, this.start);
+        int utflen2 = getUTFLen(bytes, start);
+
+        int c1 = 0;
+        int c2 = 0;
+
+        int s1Start = this.start + 2;
+        int s2Start = start + 2;
+
+        while (c1 < utflen1 && c2 < utflen2) {
+            char ch1 = Character.toLowerCase(UTF8StringPointable.charAt(this.bytes, s1Start + c1));
+            char ch2 = Character.toLowerCase(UTF8StringPointable.charAt(bytes, s2Start + c2));
+
+            if (ch1 != ch2) {
+                return ch1 - ch2;
+            }
+            c1 += UTF8StringPointable.charSize(this.bytes, s1Start + c1);
+            c2 += UTF8StringPointable.charSize(bytes, s2Start + c2);
+        }
+        return utflen1 - utflen2;
+    }
+
+    @Override
+    public int hash() {
+        int h = 0;
+        int utflen = getUTFLen(bytes, start);
+        int sStart = start + 2;
+        int c = 0;
+
+        while (c < utflen) {
+            char ch = Character.toLowerCase(UTF8StringPointable.charAt(bytes, sStart + c));
+            h = 31 * h + ch;
+            c += UTF8StringPointable.charSize(bytes, sStart + c);
+        }
+        return h;
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index a996683..417948f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -22,6 +22,8 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.ARectangleTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.AStringTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryBooleanOrNullFunctionTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringBoolOrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringStringOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.CastRecordResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
@@ -31,6 +33,8 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedGetItemResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedLocalAvgTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericAddSubMulDivTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericRoundHalfToEven2TypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericUnaryFunctionTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedSumTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedSwitchCaseComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedUnaryMinusTypeComputer;
@@ -56,8 +60,15 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListOfAInt32TypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListOfAStringTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.QuadStringStringOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ScalarVersionOfAggregateResultType;
+import edu.uci.ics.asterix.om.typecomputer.impl.Substring2TypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.SubstringTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.TripleStringBoolOrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.TripleStringStringOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.UnaryBooleanOrNullFunctionTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.UnaryStringInt32OrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.UnaryStringOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.UnorderedListConstructorResultType;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -97,245 +108,285 @@
     private final static Map<IFunctionInfo, IFunctionInfo> scalarToAggregateFunctionMap = new HashMap<IFunctionInfo, IFunctionInfo>();
     private static final Map<IFunctionInfo, SpatialFilterKind> spatialFilterFunctions = new HashMap<IFunctionInfo, SpatialFilterKind>();
 
-    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "type-of", 1,
-            true);
+    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "type-of", 1);
     public final static FunctionIdentifier GET_HANDLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "get-handle", 2, true);
+            "get-handle", 2);
     public final static FunctionIdentifier GET_DATA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-data",
-            2, true);
+            2);
     public final static FunctionIdentifier EMBED_TYPE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "embed-type", 1, true);
+            "embed-type", 1);
 
     public final static FunctionIdentifier GET_ITEM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item",
-            2, true);
+            2);
     public final static FunctionIdentifier ANY_COLLECTION_MEMBER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "any-collection-member", 1, true);
-    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1,
-            true);
+            "any-collection-member", 1);
+    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1);
     // public final static FunctionIdentifier BAGIFY = new
     // FunctionIdentifier(ASTERIX_NS, "bagify", 1, true);
-    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1, true);
+    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1);
 
     public final static FunctionIdentifier CONCAT_NON_NULL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "concat-non-null", FunctionIdentifier.VARARGS, true);
+            "concat-non-null", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "empty-stream", 0, true);
+            "empty-stream", 0);
     public final static FunctionIdentifier NON_EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "non-empty-stream", 0, true);
+            "non-empty-stream", 0);
     public final static FunctionIdentifier ORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "ordered-list-constructor", FunctionIdentifier.VARARGS, true);
+            FunctionConstants.ASTERIX_NS, "ordered-list-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier UNORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "unordered-list-constructor", FunctionIdentifier.VARARGS, true);
+            FunctionConstants.ASTERIX_NS, "unordered-list-constructor", FunctionIdentifier.VARARGS);
 
     // records
     public final static FunctionIdentifier CLOSED_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "closed-record-constructor", FunctionIdentifier.VARARGS, true);
+            FunctionConstants.ASTERIX_NS, "closed-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier OPEN_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "open-record-constructor", FunctionIdentifier.VARARGS, true);
+            FunctionConstants.ASTERIX_NS, "open-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier RECORD_TYPE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "record-type-constructor", FunctionIdentifier.VARARGS, true);
+            FunctionConstants.ASTERIX_NS, "record-type-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-index", 2, true);
+            "field-access-by-index", 2);
     public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-name", 2, true);
+            "field-access-by-name", 2);
 
     public final static FunctionIdentifier NUMERIC_UNARY_MINUS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-unary-minus", 1, true);
+            "numeric-unary-minus", 1);
 
     public final static FunctionIdentifier NUMERIC_SUBTRACT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-subtract", 2, true);
+            "numeric-subtract", 2);
     public final static FunctionIdentifier NUMERIC_MULTIPLY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-multiply", 2, true);
+            "numeric-multiply", 2);
     public final static FunctionIdentifier NUMERIC_DIVIDE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-divide", 2, true);
+            "numeric-divide", 2);
     public final static FunctionIdentifier NUMERIC_MOD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-mod", 2, true);
+            "numeric-mod", 2);
     public final static FunctionIdentifier NUMERIC_IDIV = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-idiv", 2, true);
-    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2,
-            true);
+            "numeric-idiv", 2);
+    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
 
-    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1,
-            true);
+	public final static FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-abs", 1);
+	public final static FunctionIdentifier NUMERIC_CEILING = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-ceiling", 1);
+	public final static FunctionIdentifier NUMERIC_FLOOR = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-floor", 1);
+	public final static FunctionIdentifier NUMERIC_ROUND = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-round", 1);
+	public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 1);
+	public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN2 = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even2", 2);
+	// String funcitons
+	public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "string-equal", 2);
+	public final static FunctionIdentifier STRING_START_WITH = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "start-with", 2);
+	public final static FunctionIdentifier STRING_END_WITH = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "end-with", 2);
+	public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "matches", 2);
+	public final static FunctionIdentifier STRING_MATCHES_WITH_FLAG = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "matches2", 3);
+	public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "lowercase", 1);
+	public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "replace", 3);
+	public final static FunctionIdentifier STRING_REPLACE_WITH_FLAG = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "replace2", 4);
+	public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "string-length", 1);
+	public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "substring2", 2);
+	public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "substring-before", 2);
+	public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "substring-after", 2);
+	public final static FunctionIdentifier STRING_TO_CODEPOINT = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "string-to-codepoint", 1);
+	public final static FunctionIdentifier CODEPOINT_TO_STRING = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "codepoint-to-string", 1);
+	public final static FunctionIdentifier STRING_CONCAT = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "string-concat", 1);
+	public final static FunctionIdentifier STRING_JOIN = new FunctionIdentifier(
+			FunctionConstants.ASTERIX_NS, "string-join", 2);
+
+    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
     public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "feed-ingest", 1, true);
+            "feed-ingest", 1);
 
     public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "index-search", FunctionIdentifier.VARARGS, true);
+            "index-search", FunctionIdentifier.VARARGS);
 
     public final static FunctionIdentifier MAKE_FIELD_INDEX_HANDLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2, true);
+            FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2);
     public final static FunctionIdentifier MAKE_FIELD_NAME_HANDLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "make-field-name-handle", 1, true);
+            FunctionConstants.ASTERIX_NS, "make-field-name-handle", 1);
 
     public final static FunctionIdentifier SUBSTRING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "substring", 3, true);
-    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2, true);
+            "substring", 3);
+    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2);
     public final static FunctionIdentifier CONTAINS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "contains",
-            2, true);
+            2);
     private final static FunctionIdentifier STARTS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "starts-with", 2, true);
+            "starts-with", 2);
     private final static FunctionIdentifier ENDS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "ends-with", 2, true);
+            "ends-with", 2);
 
-    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1,
-            true);
-    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1,
-            true);
-    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1,
-            true);
-    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1,
-            true);
-    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1,
-            true);
+    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1);
+    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1);
+    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
+    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1);
+    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1);
     public final static FunctionIdentifier GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "agg-global-avg", 1, true);
+            "agg-global-avg", 1);
     public final static FunctionIdentifier LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "agg-local-avg", 1, true);
+            "agg-local-avg", 1);
 
-    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1,
-            true);
+    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
     public final static FunctionIdentifier SCALAR_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "count",
-            1, true);
-    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1,
-            true);
-    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1,
-            true);
-    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1,
-            true);
+            1);
+    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1);
+    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1);
+    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1);
     public final static FunctionIdentifier SCALAR_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "global-avg", 1, true);
+            "global-avg", 1);
     public final static FunctionIdentifier SCALAR_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-avg", 1, true);
+            "local-avg", 1);
 
     // serializable aggregate functions
     public final static FunctionIdentifier SERIAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "avg-serial", 1, true);
+            "avg-serial", 1);
     public final static FunctionIdentifier SERIAL_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "count-serial", 1, true);
+            "count-serial", 1);
     public final static FunctionIdentifier SERIAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "sum-serial", 1, true);
+            "sum-serial", 1);
     public final static FunctionIdentifier SERIAL_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "global-avg-serial", 1, true);
+            "global-avg-serial", 1);
     public final static FunctionIdentifier SERIAL_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-avg-serial", 1, true);
+            "local-avg-serial", 1);
 
-    public final static FunctionIdentifier YEAR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year", 1, true);
+    public final static FunctionIdentifier YEAR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year", 1);
 
     public final static FunctionIdentifier SCAN_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "scan-collection", 1, true);
+            "scan-collection", 1);
     public final static FunctionIdentifier SUBSET_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subset-collection", 3, true);
+            "subset-collection", 3);
 
-    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2,
-            true);
+    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
 
     // fuzzy functions:
     public final static FunctionIdentifier FUZZY_EQ = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq",
-            2, true);
+            2);
 
     public final static FunctionIdentifier PREFIX_LEN_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "prefix-len-jaccard", 2, true);
+            "prefix-len-jaccard", 2);
 
     public final static FunctionIdentifier SIMILARITY_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard", 2, true);
+            "similarity-jaccard", 2);
     public final static FunctionIdentifier SIMILARITY_JACCARD_CHECK = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-check", 3, true);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-check", 3);
+    public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted", 2);
+    public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED_CHECK = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted-check", 3);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix", 6, true);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix", 6);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX_CHECK = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix-check", 6, true);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix-check", 6);
 
     public final static FunctionIdentifier EDIT_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "edit-distance", 2, true);
+            "edit-distance", 2);
     public final static FunctionIdentifier EDIT_DISTANCE_CHECK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "edit-distance-check", 3, true);
+            "edit-distance-check", 3);
+    public final static FunctionIdentifier EDIT_DISTANCE_LIST_IS_FILTERABLE = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "edit-distance-list-is-filterable", 2);
+    public final static FunctionIdentifier EDIT_DISTANCE_STRING_IS_FILTERABLE = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "edit-distance-string-is-filterable", 4);
 
     // tokenizers:
     public final static FunctionIdentifier WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "word-tokens", 1, true);
+            "word-tokens", 1);
     public final static FunctionIdentifier HASHED_WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-word-tokens", 1, true);
+            "hashed-word-tokens", 1);
     public final static FunctionIdentifier COUNTHASHED_WORD_TOKENS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "counthashed-word-tokens", 1, true);
+            FunctionConstants.ASTERIX_NS, "counthashed-word-tokens", 1);
     public final static FunctionIdentifier GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "gram-tokens", 3, true);
+            "gram-tokens", 3);
     public final static FunctionIdentifier HASHED_GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-gram-tokens", 3, true);
+            "hashed-gram-tokens", 3);
     public final static FunctionIdentifier COUNTHASHED_GRAM_TOKENS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "counthashed-gram-tokens", 3, true);
+            FunctionConstants.ASTERIX_NS, "counthashed-gram-tokens", 3);
 
-    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0, true);
+    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0);
 
     // constructors:
     public final static FunctionIdentifier BOOLEAN_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "boolean", 1, false);
+            "boolean", 1);
     public final static FunctionIdentifier NULL_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "null", 1, false);
+            "null", 1);
     public final static FunctionIdentifier STRING_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string", 1, false);
+            "string", 1);
     public final static FunctionIdentifier INT8_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int8", 1, false);
+            "int8", 1);
     public final static FunctionIdentifier INT16_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int16", 1, false);
+            "int16", 1);
     public final static FunctionIdentifier INT32_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int32", 1, false);
+            "int32", 1);
     public final static FunctionIdentifier INT64_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int64", 1, false);
+            "int64", 1);
     public final static FunctionIdentifier FLOAT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "float", 1, false);
+            "float", 1);
     public final static FunctionIdentifier DOUBLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "double", 1, false);
+            "double", 1);
     public final static FunctionIdentifier POINT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "point", 1, false);
+            "point", 1);
     public final static FunctionIdentifier POINT3D_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "point3d", 1, false);
+            "point3d", 1);
     public final static FunctionIdentifier LINE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "line", 1, false);
+            "line", 1);
     public final static FunctionIdentifier CIRCLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "circle", 1, false);
+            "circle", 1);
     public final static FunctionIdentifier RECTANGLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "rectangle", 1, false);
+            "rectangle", 1);
     public final static FunctionIdentifier POLYGON_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "polygon", 1, false);
+            "polygon", 1);
     public final static FunctionIdentifier TIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "time", 1, false);
+            "time", 1);
     public final static FunctionIdentifier DATE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "date", 1, false);
+            "date", 1);
     public final static FunctionIdentifier DATETIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "datetime", 1, false);
+            "datetime", 1);
     public final static FunctionIdentifier DURATION_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "duration", 1, false);
+            "duration", 1);
 
     // spatial
     public final static FunctionIdentifier CREATE_POINT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-point", 2, true);
+            "create-point", 2);
     public final static FunctionIdentifier CREATE_LINE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-line", 2, true);
+            "create-line", 2);
     public final static FunctionIdentifier CREATE_POLYGON = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-polygon", FunctionIdentifier.VARARGS, true);
+            "create-polygon", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier CREATE_CIRCLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-circle", 2, true);
+            "create-circle", 2);
     public final static FunctionIdentifier CREATE_RECTANGLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-rectangle", 2, true);
+            "create-rectangle", 2);
     public final static FunctionIdentifier SPATIAL_INTERSECT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-intersect", 2, true);
+            "spatial-intersect", 2);
     public final static FunctionIdentifier SPATIAL_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-area", 1, true);
+            "spatial-area", 1);
     public final static FunctionIdentifier SPATIAL_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-distance", 2, true);
+            "spatial-distance", 2);
     public final static FunctionIdentifier CREATE_MBR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-mbr", 3, true);
+            "create-mbr", 3);
     public final static FunctionIdentifier SPATIAL_CELL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-cell", 4, true);
+            "spatial-cell", 4);
     public final static FunctionIdentifier SWITCH_CASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "switch-case", FunctionIdentifier.VARARGS, true);
-    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2,
-            true);
+            "switch-case", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2);
+
     public final static FunctionIdentifier INJECT_FAILURE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "inject-failure", 2, true);
+            "inject-failure", 2);
     public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "cast-record", 1, true);
+            "cast-record", 1);
 
     public static final FunctionIdentifier EQ = AlgebricksBuiltinFunctions.EQ;
     public static final FunctionIdentifier LE = AlgebricksBuiltinFunctions.LE;
@@ -352,10 +403,10 @@
     public static IFunctionInfo getAsterixFunctionInfo(FunctionIdentifier fid) {
         IFunctionInfo finfo = asterixFunctionIdToInfo.get(fid);
         if (finfo == null) {
-            finfo = new AsterixFunctionInfo(fid, fid.isBuiltin());
-            if (fid.isBuiltin()) {
-                asterixFunctionIdToInfo.put(fid, finfo);
-            }
+            finfo = new AsterixFunctionInfo(fid);
+            //   if (fid.isBuiltin()) {
+            asterixFunctionIdToInfo.put(fid, finfo);
+            //  }
         }
         return finfo;
     }
@@ -419,6 +470,8 @@
         add(DURATION_CONSTRUCTOR, OptionalADurationTypeComputer.INSTANCE);
         add(EDIT_DISTANCE, AInt32TypeComputer.INSTANCE);
         add(EDIT_DISTANCE_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        add(EDIT_DISTANCE_STRING_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
+        add(EDIT_DISTANCE_LIST_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
         add(EMBED_TYPE, new IResultTypeComputer() {
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
@@ -470,6 +523,32 @@
         add(NUMERIC_DIVIDE, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
         add(NUMERIC_MOD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
         add(NUMERIC_IDIV, AInt32TypeComputer.INSTANCE);
+        
+        // Xiaoyu Ma Add for new functions
+        add(NUMERIC_ABS, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        add(NUMERIC_CEILING, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        add(NUMERIC_FLOOR, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        add(NUMERIC_ROUND, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        add(NUMERIC_ROUND_HALF_TO_EVEN, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        add(NUMERIC_ROUND_HALF_TO_EVEN2, NonTaggedNumericRoundHalfToEven2TypeComputer.INSTANCE);
+        
+        add(STRING_TO_CODEPOINT, OrderedListOfAInt32TypeComputer.INSTANCE);
+        add(CODEPOINT_TO_STRING, AStringTypeComputer.INSTANCE);
+        add(STRING_CONCAT, AStringTypeComputer.INSTANCE);
+        add(SUBSTRING2, Substring2TypeComputer.INSTANCE);
+        add(STRING_LENGTH, UnaryStringInt32OrNullTypeComputer.INSTANCE);
+        add(STRING_LOWERCASE, UnaryStringOrNullTypeComputer.INSTANCE);
+        add(STRING_START_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        add(STRING_END_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        add(STRING_MATCHES, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        add(STRING_MATCHES_WITH_FLAG, TripleStringBoolOrNullTypeComputer.INSTANCE);
+        add(STRING_REPLACE, TripleStringStringOrNullTypeComputer.INSTANCE);
+        add(STRING_REPLACE_WITH_FLAG, QuadStringStringOrNullTypeComputer.INSTANCE);
+        add(SUBSTRING_BEFORE, BinaryStringStringOrNullTypeComputer.INSTANCE);
+        add(SUBSTRING_AFTER, BinaryStringStringOrNullTypeComputer.INSTANCE);
+        add(STRING_EQUAL, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        add(STRING_JOIN, AStringTypeComputer.INSTANCE);
+        
         add(OPEN_RECORD_CONSTRUCTOR, OpenRecordConstructorResultType.INSTANCE);
         add(ORDERED_LIST_CONSTRUCTOR, OrderedListConstructorResultType.INSTANCE);
         add(POINT_CONSTRUCTOR, OptionalAPointTypeComputer.INSTANCE);
@@ -494,6 +573,8 @@
         add(SERIAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
         add(SIMILARITY_JACCARD, AFloatTypeComputer.INSTANCE);
         add(SIMILARITY_JACCARD_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        add(SIMILARITY_JACCARD_SORTED, AFloatTypeComputer.INSTANCE);
+        add(SIMILARITY_JACCARD_SORTED_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
         add(SIMILARITY_JACCARD_PREFIX, AFloatTypeComputer.INSTANCE);
         add(SIMILARITY_JACCARD_PREFIX_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
         add(SPATIAL_AREA, ADoubleTypeComputer.INSTANCE);
@@ -539,7 +620,7 @@
                 }
             }
         });
-        add(SUBSTRING, AStringTypeComputer.INSTANCE);
+        add(SUBSTRING, SubstringTypeComputer.INSTANCE);
         add(SUM, NonTaggedSumTypeComputer.INSTANCE);
         add(SWITCH_CASE, NonTaggedSwitchCaseComputer.INSTANCE);
         add(REG_EXP, ABooleanTypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
index 8580d33..61e47c0 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
@@ -20,15 +20,13 @@
 public class AsterixFunctionInfo implements IFunctionInfo {
 
     private final FunctionIdentifier functionIdentifier;
-    
 
-    public AsterixFunctionInfo(String namespace, AsterixFunction asterixFunction, boolean isBuiltin) {
+    public AsterixFunctionInfo(String namespace, AsterixFunction asterixFunction) {
         this.functionIdentifier = new FunctionIdentifier(namespace, asterixFunction.getFunctionName(),
-                asterixFunction.getArity(), isBuiltin);
+                asterixFunction.getArity());
     }
 
-    public AsterixFunctionInfo(FunctionIdentifier functionIdentifier,
-            boolean isBuiltin) {
+    public AsterixFunctionInfo(FunctionIdentifier functionIdentifier) {
         this.functionIdentifier = functionIdentifier;
     }
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java
new file mode 100644
index 0000000..0615de4
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractBinaryStringTypeComputer.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+abstract public class AbstractBinaryStringTypeComputer implements IResultTypeComputer {
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().size() < 2)
+            throw new AlgebricksException("Wrong Argument Number.");        
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
+        IAType t0, t1;
+        try {
+            t0 = (IAType) env.getType(arg0);
+            t1 = (IAType) env.getType(arg1);
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        if ((t0.getTypeTag() != ATypeTag.NULL && t0.getTypeTag() != ATypeTag.STRING) ||
+            (t1.getTypeTag() != ATypeTag.NULL && t1.getTypeTag() != ATypeTag.STRING)) {
+            throw new AlgebricksException("Expects String Type.");
+        }      
+
+        return getResultType(t0, t1);
+    }    
+    
+    
+    public abstract IAType getResultType(IAType t0, IAType t1);
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java
new file mode 100644
index 0000000..0baf221
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractQuadStringTypeComputer.java
@@ -0,0 +1,50 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public abstract class AbstractQuadStringTypeComputer implements IResultTypeComputer {
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().size() < 4)
+            throw new AlgebricksException("Wrong Argument Number.");        
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
+        ILogicalExpression arg2 = fce.getArguments().get(2).getValue();  
+        ILogicalExpression arg3 = fce.getArguments().get(3).getValue();                
+        IAType t0, t1, t2, t3;
+        try {
+            t0 = (IAType) env.getType(arg0);
+            t1 = (IAType) env.getType(arg1);
+            t2 = (IAType) env.getType(arg2);    
+            t3 = (IAType) env.getType(arg3);              
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        if ((t0.getTypeTag() != ATypeTag.NULL && t0.getTypeTag() != ATypeTag.STRING) || 
+            (t1.getTypeTag() != ATypeTag.NULL && t1.getTypeTag() != ATypeTag.STRING) ||
+            (t2.getTypeTag() != ATypeTag.NULL && t2.getTypeTag() != ATypeTag.STRING) ||
+            (t3.getTypeTag() != ATypeTag.NULL && t3.getTypeTag() != ATypeTag.STRING)) {
+            throw new NotImplementedException("Expects String Type.");
+        }
+
+        return getResultType(t0, t1, t2, t3);
+    }    
+    
+    
+    public abstract IAType getResultType(IAType t0, IAType t1, IAType t2, IAType t3);    
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractTripleStringTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractTripleStringTypeComputer.java
new file mode 100644
index 0000000..429f1f0
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/AbstractTripleStringTypeComputer.java
@@ -0,0 +1,46 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public abstract class AbstractTripleStringTypeComputer implements IResultTypeComputer {
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().size() < 3)
+            throw new AlgebricksException("Wrong Argument Number.");            
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
+        ILogicalExpression arg2 = fce.getArguments().get(2).getValue();        
+        IAType t0, t1, t2;
+        try {
+            t0 = (IAType) env.getType(arg0);
+            t1 = (IAType) env.getType(arg1);
+            t2 = (IAType) env.getType(arg2);            
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        if ((t0.getTypeTag() != ATypeTag.NULL   && t0.getTypeTag() != ATypeTag.STRING) || (t1.getTypeTag() != ATypeTag.NULL &&
+            t1.getTypeTag() != ATypeTag.STRING) || (t2.getTypeTag() != ATypeTag.NULL && t2.getTypeTag() != ATypeTag.STRING)) {
+            throw new AlgebricksException("Expects String Type.");
+        }
+
+        return getResultType(t0, t1, t2);
+    }    
+    
+    
+    public abstract IAType getResultType(IAType t0, IAType t1, IAType t2);
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringBoolOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringBoolOrNullTypeComputer.java
new file mode 100644
index 0000000..fa2a4a5
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringBoolOrNullTypeComputer.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class BinaryStringBoolOrNullTypeComputer extends AbstractBinaryStringTypeComputer {
+    public static final BinaryStringBoolOrNullTypeComputer INSTANCE = new BinaryStringBoolOrNullTypeComputer();
+    private BinaryStringBoolOrNullTypeComputer() {}    
+    
+    @Override
+    public IAType getResultType(IAType t0, IAType t1) {
+        if (TypeHelper.canBeNull(t0) || TypeHelper.canBeNull(t1)) {
+            return AUnionType.createNullableType(BuiltinType.ABOOLEAN);
+        }        	
+        return BuiltinType.ABOOLEAN;
+    }    
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringStringOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringStringOrNullTypeComputer.java
new file mode 100644
index 0000000..0b24ff9
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/BinaryStringStringOrNullTypeComputer.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class BinaryStringStringOrNullTypeComputer extends AbstractBinaryStringTypeComputer {
+    
+    public static final BinaryStringStringOrNullTypeComputer INSTANCE = new BinaryStringStringOrNullTypeComputer();
+    private BinaryStringStringOrNullTypeComputer() {}
+
+    @Override
+    public IAType getResultType(IAType t0, IAType t1) {
+        if (TypeHelper.canBeNull(t0) || TypeHelper.canBeNull(t1)) {
+            return AUnionType.createNullableType(BuiltinType.ASTRING);
+        }      	
+        return BuiltinType.ASTRING;
+    }
+    
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericRoundHalfToEven2TypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericRoundHalfToEven2TypeComputer.java
new file mode 100644
index 0000000..da90a06
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericRoundHalfToEven2TypeComputer.java
@@ -0,0 +1,98 @@
+/*
+ * Numeric round half to even
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class NonTaggedNumericRoundHalfToEven2TypeComputer implements IResultTypeComputer {
+
+    public static final NonTaggedNumericRoundHalfToEven2TypeComputer INSTANCE =
+            new NonTaggedNumericRoundHalfToEven2TypeComputer();
+
+    private NonTaggedNumericRoundHalfToEven2TypeComputer() {
+    }
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().size() < 2)
+        	throw new AlgebricksException("Argument number invalid.");
+        
+        ILogicalExpression arg1 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = fce.getArguments().get(1).getValue();
+        
+        IAType t1 = (IAType) env.getType(arg1);
+        IAType t2 = (IAType) env.getType(arg2);
+        
+        List<IAType> unionList = new ArrayList<IAType>();
+        unionList.add(BuiltinType.ANULL);        
+
+        ATypeTag tag1, tag2;
+        if (t1.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t1))
+            tag1 = ((AUnionType) t1).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag1 = t1.getTypeTag();
+
+        if (t2.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t2))
+            tag2 = ((AUnionType) t2).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag2 = t2.getTypeTag();
+        
+        switch(tag2) {
+	        case INT8:
+	        case INT16:
+	        case INT32:
+	        case INT64:
+	            break;
+	        default:
+	            throw new AlgebricksException("Argument $precision cannot be type " + t2.getTypeName());
+        }        
+        
+        switch (tag1) {
+            case INT8:
+                unionList.add(BuiltinType.AINT8);
+                break;                
+            case INT16:
+                unionList.add(BuiltinType.AINT16);
+                break;                 
+            case INT32:
+                unionList.add(BuiltinType.AINT32);
+                break;                 
+            case INT64:
+                unionList.add(BuiltinType.AINT64);
+                break;                 
+            case FLOAT:
+                unionList.add(BuiltinType.AFLOAT);
+                break;                 
+            case DOUBLE:
+                unionList.add(BuiltinType.ADOUBLE);
+                break;
+            case NULL:
+                return BuiltinType.ANULL;
+            default: {
+                throw new NotImplementedException("Arithmetic operations are not implemented for " + t1.getTypeName());
+            }
+        }
+
+        return new AUnionType(unionList, "NumericFuncionsResult");
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericUnaryFunctionTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericUnaryFunctionTypeComputer.java
new file mode 100644
index 0000000..454104b
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericUnaryFunctionTypeComputer.java
@@ -0,0 +1,79 @@
+/*
+ * Numeric Unary Functions like abs
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class NonTaggedNumericUnaryFunctionTypeComputer implements IResultTypeComputer {
+
+    private static final String errMsg = "Arithmetic operations are not implemented for ";
+    public static final NonTaggedNumericUnaryFunctionTypeComputer INSTANCE =
+            new NonTaggedNumericUnaryFunctionTypeComputer();
+
+    private NonTaggedNumericUnaryFunctionTypeComputer() {
+    }
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().isEmpty())
+            throw new AlgebricksException("Wrong Argument Number.");
+        
+        ILogicalExpression arg1 = fce.getArguments().get(0).getValue();
+
+        IAType t = (IAType) env.getType(arg1);
+        ATypeTag tag = t.getTypeTag();
+
+        if (tag == ATypeTag.UNION
+                && NonTaggedFormatUtil.isOptionalField((AUnionType) env.getType(arg1))) {
+            return (IAType) env.getType(arg1);
+        }
+        
+        List<IAType> unionList = new ArrayList<IAType>();
+        unionList.add(BuiltinType.ANULL);
+        switch (tag) {
+            case INT8:
+                unionList.add(BuiltinType.AINT8);
+                break;                
+            case INT16:
+                unionList.add(BuiltinType.AINT16);
+                break;                 
+            case INT32:
+                unionList.add(BuiltinType.AINT32);
+                break;                 
+            case INT64:
+                unionList.add(BuiltinType.AINT64);
+                break;                 
+            case FLOAT:
+                unionList.add(BuiltinType.AFLOAT);
+                break;                 
+            case DOUBLE:
+                unionList.add(BuiltinType.ADOUBLE);
+                break;
+            case NULL:
+                return BuiltinType.ANULL;
+            default: {
+                throw new NotImplementedException(errMsg + t.getTypeName());
+            }
+        }
+
+        return new AUnionType(unionList, "NumericUnaryFuncionsResult");
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/QuadStringStringOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/QuadStringStringOrNullTypeComputer.java
new file mode 100644
index 0000000..876df7a
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/QuadStringStringOrNullTypeComputer.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class QuadStringStringOrNullTypeComputer  extends AbstractQuadStringTypeComputer {
+    public static final QuadStringStringOrNullTypeComputer INSTANCE = new QuadStringStringOrNullTypeComputer();
+    private QuadStringStringOrNullTypeComputer() {}
+
+    @Override
+    public IAType getResultType(IAType t0, IAType t1, IAType t2, IAType t3) {
+        if (TypeHelper.canBeNull(t0) || TypeHelper.canBeNull(t1) || 
+        	TypeHelper.canBeNull(t2) || TypeHelper.canBeNull(t3)) {
+            return AUnionType.createNullableType(BuiltinType.ASTRING);
+        }      	
+        return BuiltinType.ASTRING;
+    }
+    
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/Substring2TypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/Substring2TypeComputer.java
new file mode 100644
index 0000000..c5d224c
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/Substring2TypeComputer.java
@@ -0,0 +1,63 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+
+public class Substring2TypeComputer implements IResultTypeComputer {
+    public static final Substring2TypeComputer INSTANCE = new Substring2TypeComputer();
+    
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().size() < 2)
+            throw new AlgebricksException("Wrong Argument Number.");            
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();       
+        IAType t0, t1;
+        try {
+            t0 = (IAType) env.getType(arg0);
+            t1 = (IAType) env.getType(arg1);                  
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        
+        ATypeTag tag0, tag1;
+        if (t0.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t0))
+            tag0 = ((AUnionType) t0).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag0 = t0.getTypeTag();
+        
+        if (t1.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t1))
+            tag1 = ((AUnionType) t1).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag1 = t1.getTypeTag();        
+        
+        if(tag0 != ATypeTag.NULL && tag0 != ATypeTag.STRING) {
+            throw new AlgebricksException("First argument should be String Type.");
+        }
+        
+        if(tag1 != ATypeTag.NULL && 
+           tag1 != ATypeTag.INT8 && 
+           tag1 != ATypeTag.INT16 && 
+           tag1 != ATypeTag.INT32 && 
+           tag1 != ATypeTag.INT64) {
+            throw new AlgebricksException("Second argument should be integer Type.");
+        }
+
+        return BuiltinType.ASTRING;
+    }   
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/SubstringTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/SubstringTypeComputer.java
new file mode 100644
index 0000000..b016de8
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/SubstringTypeComputer.java
@@ -0,0 +1,71 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class SubstringTypeComputer implements IResultTypeComputer {
+    public static final SubstringTypeComputer INSTANCE = new SubstringTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if (fce.getArguments().size() < 3)
+            throw new AlgebricksException("Wrong Argument Number.");
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        ILogicalExpression arg1 = fce.getArguments().get(1).getValue();
+        ILogicalExpression arg2 = fce.getArguments().get(2).getValue();
+        IAType t0, t1, t2;
+        try {
+            t0 = (IAType) env.getType(arg0);
+            t1 = (IAType) env.getType(arg1);
+            t2 = (IAType) env.getType(arg2);
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+
+        ATypeTag tag0, tag1, tag2;
+        if (t0.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t0))
+            tag0 = ((AUnionType) t0).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag0 = t0.getTypeTag();
+
+        if (t1.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t1))
+            tag1 = ((AUnionType) t1).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag1 = t1.getTypeTag();
+
+        if (t2.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) t2))
+            tag2 = ((AUnionType) t2).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST)
+                    .getTypeTag();
+        else
+            tag2 = t2.getTypeTag();
+
+        if (tag0 != ATypeTag.NULL && tag0 != ATypeTag.STRING) {
+            throw new AlgebricksException("First argument should be String Type.");
+        }
+
+        if (tag1 != ATypeTag.NULL && tag1 != ATypeTag.INT8 && tag1 != ATypeTag.INT16 && tag1 != ATypeTag.INT32
+                && tag1 != ATypeTag.INT64) {
+            throw new AlgebricksException("Second argument should be integer Type.");
+        }
+
+        if (tag2 != ATypeTag.NULL && tag2 != ATypeTag.INT8 && tag2 != ATypeTag.INT16 && tag2 != ATypeTag.INT32
+                && tag2 != ATypeTag.INT64) {
+            throw new AlgebricksException("Third argument should be integer Type.");
+        }
+
+        return BuiltinType.ASTRING;
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringBoolOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringBoolOrNullTypeComputer.java
new file mode 100644
index 0000000..0bac1df
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringBoolOrNullTypeComputer.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class TripleStringBoolOrNullTypeComputer  extends AbstractTripleStringTypeComputer {
+    public static final TripleStringBoolOrNullTypeComputer INSTANCE = new TripleStringBoolOrNullTypeComputer();
+    private TripleStringBoolOrNullTypeComputer() {}    
+    @Override
+    public IAType getResultType(IAType t0, IAType t1, IAType t2) {
+        if (TypeHelper.canBeNull(t0) || TypeHelper.canBeNull(t1) || TypeHelper.canBeNull(t2)) {
+            return AUnionType.createNullableType(BuiltinType.ABOOLEAN);
+        }    	
+        return BuiltinType.ABOOLEAN;
+    }
+    
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringStringOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringStringOrNullTypeComputer.java
new file mode 100644
index 0000000..10154b0
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/TripleStringStringOrNullTypeComputer.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class TripleStringStringOrNullTypeComputer  extends AbstractTripleStringTypeComputer {
+    public static final TripleStringStringOrNullTypeComputer INSTANCE = new TripleStringStringOrNullTypeComputer();
+    private TripleStringStringOrNullTypeComputer() {}
+
+    @Override
+    public IAType getResultType(IAType t0, IAType t1, IAType t2) {
+        if (TypeHelper.canBeNull(t0) || TypeHelper.canBeNull(t1) || TypeHelper.canBeNull(t2)) {
+            return AUnionType.createNullableType(BuiltinType.ASTRING);
+        }      	
+        return BuiltinType.ASTRING;
+    } 
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java
new file mode 100644
index 0000000..bcd45a7
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java
@@ -0,0 +1,59 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class UnaryStringInt32OrNullTypeComputer implements IResultTypeComputer  {   
+    
+    public static final UnaryStringInt32OrNullTypeComputer INSTANCE = new UnaryStringInt32OrNullTypeComputer();
+    private UnaryStringInt32OrNullTypeComputer() {}
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().isEmpty())
+            throw new AlgebricksException("Wrong Argument Number.");        
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        IAType t0;
+        try {
+            t0 = (IAType) env.getType(arg0);
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        if (t0.getTypeTag() != ATypeTag.NULL &&
+            t0.getTypeTag() != ATypeTag.STRING ) {
+            throw new NotImplementedException("Expects String Type.");
+        }     
+        
+        List<IAType> unionList = new ArrayList<IAType>();
+        unionList.add(BuiltinType.ANULL);
+        if(t0.getTypeTag() == ATypeTag.NULL) {
+            return BuiltinType.ANULL;
+        }
+        
+        if(t0.getTypeTag() == ATypeTag.STRING) {
+            unionList.add(BuiltinType.AINT32);
+        }        
+        
+        return new AUnionType(unionList, "String-length-Result");
+    }      
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringOrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringOrNullTypeComputer.java
new file mode 100644
index 0000000..d26a43f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringOrNullTypeComputer.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class UnaryStringOrNullTypeComputer implements IResultTypeComputer  {   
+    
+    public static final UnaryStringOrNullTypeComputer INSTANCE = new UnaryStringOrNullTypeComputer();
+    private UnaryStringOrNullTypeComputer() {}
+    
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        if(fce.getArguments().isEmpty())
+            throw new AlgebricksException("Wrong Argument Number.");        
+        ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
+        IAType t0;
+        try {
+            t0 = (IAType) env.getType(arg0);
+        } catch (AlgebricksException e) {
+            throw new AlgebricksException(e);
+        }
+        
+        if (TypeHelper.canBeNull(t0)) {
+            return AUnionType.createNullableType(BuiltinType.ASTRING);
+        }            
+        
+        if (t0.getTypeTag() == ATypeTag.NULL)
+        	return BuiltinType.ANULL;
+        
+        if(t0.getTypeTag() == ATypeTag.STRING) 
+        	return BuiltinType.ASTRING;
+        
+        throw new AlgebricksException("Expects String Type.");        
+    }      
+}
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 2953966..1cf6ba7 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
@@ -2,7 +2,9 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import edu.uci.ics.asterix.common.annotations.IRecordTypeAnnotation;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -15,13 +17,17 @@
     private String[] fieldNames;
     private IAType[] fieldTypes;
     private boolean isOpen;
-    private transient List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+    private final List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+    private final Map<String, Integer> typeMap = new HashMap<String, Integer>();
 
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
         super(typeName);
         this.fieldNames = fieldNames;
         this.fieldTypes = fieldTypes;
         this.isOpen = isOpen;
+        for (int i = 0; i < fieldNames.length; i++) {
+            typeMap.put(fieldNames[i], i);
+        }
     }
 
     public final String[] getFieldNames() {
@@ -76,6 +82,10 @@
         return -1;
     }
 
+    public IAType getFieldType(String fieldName) {
+        return fieldTypes[typeMap.get(fieldName)];
+    }
+
     @Override
     public String getDisplayName() {
         return "ARecord";
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index 05aa2ea..8ddcb3f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -13,8 +13,7 @@
 public class ListifyAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ListifyAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
index c8cfbbe..cfbfa4e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
@@ -35,8 +35,7 @@
 public class SerializableAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg-serial", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg-serial", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SerializableAvgAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
index b030d11..98e86d2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
@@ -26,7 +26,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "count-serial",
-            1, true);
+            1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SerializableCountAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
index d281750..e4c4931 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
@@ -43,7 +43,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "global-avg-serial", 1, true);
+            "global-avg-serial", 1);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_RECORD_TYPE_TAG = ATypeTag.RECORD.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
index c04dcf0..f5c9538 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
@@ -47,7 +47,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-avg-serial", 1, true);
+            "local-avg-serial", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SerializableLocalAvgAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
index cbc5358..e689ab4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
@@ -39,8 +39,7 @@
 public class SerializableSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum-serial", 1,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum-serial", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SerializableSumAggregateDescriptor();
@@ -106,8 +105,8 @@
                         inputVal.reset();
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     metInt8s = true;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
index 078e32d..ecdd194 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
@@ -47,8 +47,7 @@
 public class AvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new AvgAggregateDescriptor();
@@ -119,8 +118,8 @@
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
                             ++count;
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     byte val = AInt8SerializerDeserializer.getByte(inputVal.getByteArray(), 1);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
index f0f7602..040e2d0 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
@@ -26,8 +26,7 @@
 public class CountAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CountAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
index 172cd62..e86b2bc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
@@ -44,7 +44,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-avg",
-            1, true);
+            1);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_RECORD_TYPE_TAG = ATypeTag.RECORD.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
index f58a084..fa05d4b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
@@ -48,7 +48,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-avg",
-            1, true);
+            1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LocalAvgAggregateDescriptor();
@@ -120,8 +120,8 @@
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
                             ++count;
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     byte val = AInt8SerializerDeserializer.getByte(inputVal.getByteArray(), 1);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
index 2fcca4c..e409910 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
@@ -38,8 +38,7 @@
 public class MaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new MaxAggregateDescriptor();
@@ -105,8 +104,8 @@
                         inputVal.reset();
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     metInt8s = true;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MinAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
index 8abf8b5..af54c14 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
@@ -38,8 +38,7 @@
 public class MinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new MinAggregateDescriptor();
@@ -105,8 +104,8 @@
                         inputVal.reset();
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     metInt8s = true;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/SumAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
index 197878b..5e45432 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
@@ -40,8 +40,7 @@
 public class SumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SumAggregateDescriptor();
@@ -97,8 +96,8 @@
                         inputVal.reset();
                         eval.evaluate(tuple);
                         if (inputVal.getLength() > 0) {
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(inputVal.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal
+                                    .getByteArray()[0]);
                             switch (typeTag) {
                                 case INT8: {
                                     metInt8s = true;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
index a48a99f..9d235e5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
@@ -24,7 +24,7 @@
     private static final long serialVersionUID = 1L;
 
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "non-empty-stream", 0, true);
+            "non-empty-stream", 0);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NonEmptyStreamAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilter.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilter.java
index da006bd..3f4f5ca 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilter.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilter.java
@@ -19,21 +19,21 @@
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
 
 public class AsterixTupleFilter implements ITupleFilter {
-
     private final IBinaryBooleanInspector boolInspector;
     private final IScalarEvaluator eval;
     private final IPointable p = VoidPointable.FACTORY.createPointable();
 
-    public AsterixTupleFilter(IScalarEvaluatorFactory evalFactory, IBinaryBooleanInspector boolInspector)
-            throws AlgebricksException {
+    public AsterixTupleFilter(IHyracksTaskContext ctx, IScalarEvaluatorFactory evalFactory,
+            IBinaryBooleanInspector boolInspector) throws AlgebricksException {
         this.boolInspector = boolInspector;
-        eval = evalFactory.createScalarEvaluator();
+        eval = evalFactory.createScalarEvaluator(ctx);
     }
 
     @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilterFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilterFactory.java
index 1495b72..1dd7ba5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilterFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/base/AsterixTupleFilterFactory.java
@@ -16,8 +16,9 @@
 package edu.uci.ics.asterix.runtime.base;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
 
@@ -25,18 +26,18 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final IBinaryBooleanInspector boolInspector;
+    private final IBinaryBooleanInspectorFactory boolInspectorFactory;
     private final IScalarEvaluatorFactory evalFactory;
 
-    public AsterixTupleFilterFactory(IScalarEvaluatorFactory evalFactory, IBinaryBooleanInspector boolInspector)
-            throws AlgebricksException {
+    public AsterixTupleFilterFactory(IScalarEvaluatorFactory evalFactory,
+            IBinaryBooleanInspectorFactory boolInspectorFactory) throws AlgebricksException {
         this.evalFactory = evalFactory;
-        this.boolInspector = boolInspector;
+        this.boolInspectorFactory = boolInspectorFactory;
     }
 
     @Override
-    public ITupleFilter createTupleFilter() throws Exception {
-        return new AsterixTupleFilter(evalFactory, boolInspector);
+    public ITupleFilter createTupleFilter(IHyracksTaskContext ctx) throws Exception {
+        return new AsterixTupleFilter(ctx, evalFactory, boolInspectorFactory.createBinaryBooleanInspector(ctx));
     }
 
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
index 0b9aca8..efae5f9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
@@ -17,6 +17,9 @@
     protected int startOff = -1;
     protected IBinaryComparator cmp;
 
+    // Ignore case for strings. Defaults to true.
+    protected final boolean ignoreCase = true;
+
     @Override
     public int compare(IListIterator cmpIter) {
         return cmp.compare(data, pos, -1, cmpIter.getData(), cmpIter.getPos(), -1);
@@ -80,7 +83,16 @@
                 break;
             }
             case STRING: {
-                cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
+                if (ignoreCase) {
+                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE
+                            .createBinaryComparator();
+                } else {
+                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
+                }
+                break;
+            }
+            default: {
+                cmp = null;
                 break;
             }
         }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
index 606b47f..b1d3cab 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
@@ -7,10 +7,6 @@
 import edu.uci.ics.asterix.builders.OrderedListBuilder;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.fuzzyjoin.IntArray;
-import edu.uci.ics.fuzzyjoin.tokenizer.IBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.IToken;
-import edu.uci.ics.fuzzyjoin.tokenizer.NGramUTF8StringBinaryTokenizer;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -19,6 +15,10 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class GramTokensEvaluator implements ICopyEvaluator {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
new file mode 100644
index 0000000..696f74f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
@@ -0,0 +1,114 @@
+package edu.uci.ics.asterix.runtime.evaluators.common;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.builders.IAOrderedListBuilder;
+import edu.uci.ics.asterix.builders.OrderedListBuilder;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.functions.BinaryHashMap.BinaryEntry;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class SimilarityJaccardCheckEvaluator extends SimilarityJaccardEvaluator {
+
+    protected final ICopyEvaluator jaccThreshEval;
+    protected float jaccThresh = -1f;
+
+    protected IAOrderedListBuilder listBuilder;
+    protected ArrayBackedValueStorage inputVal;
+    @SuppressWarnings("unchecked")
+    protected final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+    protected final AOrderedListType listType = new AOrderedListType(BuiltinType.ANY, "list");
+
+    public SimilarityJaccardCheckEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
+            throws AlgebricksException {
+        super(args, output);
+        jaccThreshEval = args[2].createEvaluator(argOut);
+        listBuilder = new OrderedListBuilder();
+        inputVal = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    protected void runArgEvals(IFrameTupleReference tuple) throws AlgebricksException {
+        super.runArgEvals(tuple);
+        int jaccThreshStart = argOut.getLength();
+        jaccThreshEval.evaluate(tuple);
+        jaccThresh = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), jaccThreshStart
+                + TYPE_INDICATOR_SIZE);
+    }
+
+    @Override
+    protected int probeHashMap(AbstractAsterixListIterator probeIter, int buildListSize, int probeListSize) {
+        // Apply length filter.
+        int lengthLowerBound = (int) Math.ceil(jaccThresh * probeListSize);
+        if ((lengthLowerBound > buildListSize) || (buildListSize > (int) Math.floor(1.0f / jaccThresh * probeListSize))) {
+            return -1;
+        }
+        // Probe phase: Probe items from second list, and compute intersection size.
+        int intersectionSize = 0;
+        int probeListCount = 0;
+        int minUnionSize = probeListSize;
+        while (probeIter.hasNext()) {
+            probeListCount++;
+            byte[] buf = probeIter.getData();
+            int off = probeIter.getPos();
+            int len = getItemLen(buf, off);
+            keyEntry.set(buf, off, len);            
+            BinaryEntry entry = hashMap.get(keyEntry);
+            if (entry != null) {
+                // Increment second value.
+                int firstValInt = IntegerPointable.getInteger(buf, 0);
+                // Irrelevant for the intersection size.
+                if (firstValInt == 0) {
+                    continue;
+                }
+                int secondValInt = IntegerPointable.getInteger(buf, 4);
+                // Subtract old min value.
+                intersectionSize -= (firstValInt < secondValInt) ? firstValInt : secondValInt;
+                secondValInt++;
+                // Add new min value.
+                intersectionSize += (firstValInt < secondValInt) ? firstValInt : secondValInt;
+                IntegerPointable.setInteger(entry.buf, 0, secondValInt);            
+            } else {
+                // Could not find element in other set. Increase min union size by 1.
+                minUnionSize++;
+                // Check whether jaccThresh can still be satisfied if there was a mismatch.
+                int maxIntersectionSize = intersectionSize + (probeListSize - probeListCount);
+                int lowerBound = (int) Math.floor(jaccThresh * minUnionSize);
+                if (maxIntersectionSize < lowerBound) {
+                    // Cannot satisfy jaccThresh.
+                    return -1;
+                }
+            }
+            probeIter.next();
+        }
+        return intersectionSize;
+    }
+    
+    @Override
+    protected void writeResult(float jacc) throws IOException {
+        listBuilder.reset(listType);
+        boolean matches = (jacc < jaccThresh) ? false : true;
+        inputVal.reset();
+        booleanSerde.serialize(matches ? ABoolean.TRUE : ABoolean.FALSE, inputVal.getDataOutput());
+        listBuilder.addItem(inputVal);
+
+        inputVal.reset();
+        aFloat.setValue((matches) ? jacc : 0.0f);
+        floatSerde.serialize(aFloat, inputVal.getDataOutput());
+        listBuilder.addItem(inputVal);
+
+        listBuilder.write(out, true);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
index a73cc72..672fb39 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
@@ -2,28 +2,38 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
 
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryHashFunctionFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AMutableFloat;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.fuzzyjoin.similarity.SimilarityMetricJaccard;
+import edu.uci.ics.asterix.runtime.evaluators.functions.BinaryHashMap;
+import edu.uci.ics.asterix.runtime.evaluators.functions.BinaryHashMap.BinaryEntry;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-// assumes that both arguments are sorted by the same ordering
-
 public class SimilarityJaccardEvaluator implements ICopyEvaluator {
 
-    // assuming type indicator in serde format
-    protected final int typeIndicatorSize = 1;
+    // Parameters for hash table.
+    protected final int TABLE_SIZE = 100;
+    protected final int TABLE_FRAME_SIZE = 32768;
+
+    // Assuming type indicator in serde format.
+    protected final int TYPE_INDICATOR_SIZE = 1;
 
     protected final DataOutput out;
     protected final ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
@@ -38,32 +48,46 @@
     protected AbstractAsterixListIterator firstListIter;
     protected AbstractAsterixListIterator secondListIter;
 
-    protected final SimilarityMetricJaccard jaccard = new SimilarityMetricJaccard();
     protected final AMutableFloat aFloat = new AMutableFloat(0);
     @SuppressWarnings("unchecked")
     protected final ISerializerDeserializer<AFloat> floatSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.AFLOAT);
 
-    protected ATypeTag itemTypeTag;
-
     protected ATypeTag firstTypeTag;
     protected ATypeTag secondTypeTag;
     protected int firstStart = -1;
     protected int secondStart = -1;
     protected float jaccSim = 0.0f;
+    protected ATypeTag itemTypeTag;
 
-    public SimilarityJaccardEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output) throws AlgebricksException {
+    protected BinaryHashMap hashMap;
+    protected BinaryEntry keyEntry = new BinaryEntry();
+    protected BinaryEntry valEntry = new BinaryEntry();
+
+    // Ignore case for strings. Defaults to true.
+    protected final boolean ignoreCase = true;
+
+    public SimilarityJaccardEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
+            throws AlgebricksException {
         out = output.getDataOutput();
         firstOrdListEval = args[0].createEvaluator(argOut);
         secondOrdListEval = args[1].createEvaluator(argOut);
+        byte[] emptyValBuf = new byte[8];
+        Arrays.fill(emptyValBuf, (byte) 0);
+        valEntry.set(emptyValBuf, 0, 8);
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
         runArgEvals(tuple);
-        if (!checkArgTypes(firstTypeTag, secondTypeTag))
+        if (!checkArgTypes(firstTypeTag, secondTypeTag)) {
             return;
-        jaccSim = computeResult(argOut.getByteArray(), firstStart, secondStart, firstTypeTag);
+        }
+        if (prepareLists(argOut.getByteArray(), firstStart, secondStart, firstTypeTag)) {
+            jaccSim = computeResult(argOut.getByteArray(), firstStart, secondStart, firstTypeTag);
+        } else {
+            jaccSim = 0.0f;
+        }
         try {
             writeResult(jaccSim);
         } catch (IOException e) {
@@ -83,26 +107,161 @@
         secondTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[secondStart]);
     }
 
-    protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
+    protected boolean prepareLists(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
             throws AlgebricksException {
         firstListIter.reset(bytes, firstStart);
         secondListIter.reset(bytes, secondStart);
         // Check for special case where one of the lists is empty, since list
         // types won't match.
         if (firstListIter.size() == 0 || secondListIter.size() == 0) {
-            try {
-                writeResult(0.0f);
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
+            return false;
+        }
+        if (firstTypeTag == ATypeTag.ANY || secondTypeTag == ATypeTag.ANY) {
+            throw new AlgebricksException("\n Jaccard can only be called on homogenous lists");
+        }
+        // TODO: Check item types are compatible.
+        itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[firstStart + 1]);
+        return true;
+    }
+
+    protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
+            throws AlgebricksException {
+        setHashMap(bytes, firstStart, secondStart);
+        // We will subtract the intersection size later to get the real union size.
+        int firstListSize = firstListIter.size();
+        int secondListSize = secondListIter.size();
+        int unionSize = firstListSize + secondListSize;
+        // Choose smaller list as build, and larger one as probe.
+        AbstractAsterixListIterator buildList = (firstListSize < secondListSize) ? firstListIter : secondListIter;
+        AbstractAsterixListIterator probeList = (buildList == firstListIter) ? secondListIter : firstListIter;
+        int buildListSize = (buildList == firstListIter) ? firstListSize : secondListSize;
+        int probeListSize = (probeList == firstListIter) ? firstListSize : secondListSize;
+
+        buildHashMap(buildList);
+        int intersectionSize = probeHashMap(probeList, buildListSize, probeListSize);
+        // Special indicator for the "check" version of jaccard.
+        if (intersectionSize < 0) {
+            return -1;
+        }
+        unionSize -= intersectionSize;
+        return (float) intersectionSize / (float) unionSize;
+    }
+
+    protected void buildHashMap(AbstractAsterixListIterator buildIter) {
+        // Build phase: Add items into hash map, starting with first list.
+        // Value in map is a pair of integers. Set first integer to 1.
+        IntegerPointable.setInteger(valEntry.buf, 0, 1);
+        while (buildIter.hasNext()) {
+            byte[] buf = buildIter.getData();
+            int off = buildIter.getPos();
+            int len = getItemLen(buf, off);
+            keyEntry.set(buf, off, len);
+            BinaryEntry entry = hashMap.put(keyEntry, valEntry);
+            if (entry != null) {
+                // Increment value.
+                int firstValInt = IntegerPointable.getInteger(entry.buf, entry.off);
+                IntegerPointable.setInteger(entry.buf, entry.off, firstValInt + 1);
+            }
+            buildIter.next();
+        }
+    }
+
+    protected int probeHashMap(AbstractAsterixListIterator probeIter, int probeListSize, int buildListSize) {
+        // Probe phase: Probe items from second list, and compute intersection size.
+        int intersectionSize = 0;
+        while (probeIter.hasNext()) {
+            byte[] buf = probeIter.getData();
+            int off = probeIter.getPos();
+            int len = getItemLen(buf, off);
+            keyEntry.set(buf, off, len);
+            BinaryEntry entry = hashMap.get(keyEntry);
+            if (entry != null) {
+                // Increment second value.
+                int firstValInt = IntegerPointable.getInteger(entry.buf, entry.off);
+                // Irrelevant for the intersection size.
+                if (firstValInt == 0) {
+                    continue;
+                }
+                int secondValInt = IntegerPointable.getInteger(entry.buf, entry.off + 4);
+                // Subtract old min value.
+                intersectionSize -= (firstValInt < secondValInt) ? firstValInt : secondValInt;
+                secondValInt++;
+                // Add new min value.
+                intersectionSize += (firstValInt < secondValInt) ? firstValInt : secondValInt;
+                IntegerPointable.setInteger(entry.buf, entry.off + 4, secondValInt);
+            }
+            probeIter.next();
+        }
+        return intersectionSize;
+    }
+
+    protected void setHashMap(byte[] bytes, int firstStart, int secondStart) {
+        if (hashMap != null) {
+            hashMap.clear();
+            return;
+        }
+        IBinaryHashFunction hashFunc = null;
+        IBinaryComparator cmp = null;
+        switch (itemTypeTag) {
+            case INT32: {
+                hashFunc = AqlBinaryHashFunctionFactoryProvider.INTEGER_POINTABLE_INSTANCE.createBinaryHashFunction();
+                cmp = AqlBinaryComparatorFactoryProvider.INTEGER_POINTABLE_INSTANCE.createBinaryComparator();
+                break;
+            }
+            case FLOAT: {
+                hashFunc = AqlBinaryHashFunctionFactoryProvider.FLOAT_POINTABLE_INSTANCE.createBinaryHashFunction();
+                cmp = AqlBinaryComparatorFactoryProvider.FLOAT_POINTABLE_INSTANCE.createBinaryComparator();
+                break;
+            }
+            case DOUBLE: {
+                hashFunc = AqlBinaryHashFunctionFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryHashFunction();
+                cmp = AqlBinaryComparatorFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryComparator();
+                break;
+            }
+            case STRING: {
+                if (ignoreCase) {
+                    // Ignore case in comparisons and hashing.
+                    hashFunc = AqlBinaryHashFunctionFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE
+                            .createBinaryHashFunction();
+                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE
+                            .createBinaryComparator();
+                } else {
+                    hashFunc = AqlBinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
+                            .createBinaryHashFunction();
+                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
+                }
+                break;
+            }
+            default: {
+                break;
             }
         }
-        if (firstTypeTag == ATypeTag.ANY || secondTypeTag == ATypeTag.ANY)
-            throw new AlgebricksException("\n Jaccard can only be called on homogenous lists");
-        return jaccard.getSimilarity(firstListIter, secondListIter);
+        hashMap = new BinaryHashMap(TABLE_SIZE, TABLE_FRAME_SIZE, hashFunc, cmp);
+    }
+
+    protected int getItemLen(byte[] bytes, int itemOff) {
+        switch (itemTypeTag) {
+            case INT32: {
+                return 4;
+            }
+            case FLOAT: {
+                return 4;
+            }
+            case DOUBLE: {
+                return 8;
+            }
+            case STRING: {
+                // 2 bytes for the UTF8 len, plus the string data.
+                return 2 + UTF8StringPointable.getUTFLength(bytes, itemOff);
+            }
+            default: {
+                return -1;
+            }
+        }
     }
 
     protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws AlgebricksException {
-        // jaccard between null and anything else is 0
+        // Jaccard between null and anything else is 0
         if (typeTag1 == ATypeTag.NULL || typeTag2 == ATypeTag.NULL) {
             try {
                 writeResult(0.0f);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java
new file mode 100644
index 0000000..f994448
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedCheckEvaluator.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.runtime.evaluators.common;
+
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.fuzzyjoin.similarity.SimilarityMetricJaccard;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+public class SimilarityJaccardSortedCheckEvaluator extends SimilarityJaccardCheckEvaluator {
+
+    protected final SimilarityMetricJaccard jaccard = new SimilarityMetricJaccard();
+    
+    public SimilarityJaccardSortedCheckEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
+            throws AlgebricksException {
+        super(args, output);
+    }
+
+    @Override
+    protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
+            throws AlgebricksException {
+        return jaccard.getSimilarity(firstListIter, secondListIter, jaccThresh);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java
new file mode 100644
index 0000000..a27d8a4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardSortedEvaluator.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.runtime.evaluators.common;
+
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.fuzzyjoin.similarity.SimilarityMetricJaccard;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+// Assumes that both arguments are sorted by the same ordering.
+public class SimilarityJaccardSortedEvaluator extends SimilarityJaccardEvaluator {
+
+    protected final SimilarityMetricJaccard jaccard = new SimilarityMetricJaccard();
+    
+	public SimilarityJaccardSortedEvaluator(ICopyEvaluatorFactory[] args,
+			IDataOutputProvider output) throws AlgebricksException {
+		super(args, output);
+	}
+
+    protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
+            throws AlgebricksException {
+        return jaccard.getSimilarity(firstListIter, secondListIter);
+    }
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
index 377a116..e52757b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
@@ -7,15 +7,15 @@
 import edu.uci.ics.asterix.builders.OrderedListBuilder;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.fuzzyjoin.IntArray;
-import edu.uci.ics.fuzzyjoin.tokenizer.IBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.IToken;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
 
 public class WordTokensEvaluator implements ICopyEvaluator {
     protected final DataOutput out;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
index 1617dc0..12a1b9a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
@@ -26,8 +26,7 @@
 public class ABooleanConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "boolean", 1,
-            false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "boolean", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
index dd4f9ef..2eb9c20 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
@@ -26,7 +26,7 @@
 public class ACircleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "circle", 1, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "circle", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 148dcb9..5ae1465 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -28,7 +28,7 @@
 public class ADateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index f077da9..2e1d0ca 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -27,8 +27,7 @@
 public class ADateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1,
-            false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
index eea58651..f0aed55 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
@@ -27,8 +27,7 @@
 public class ADoubleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "double", 1,
-            false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "double", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index 0486c04..24934b1 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -26,8 +26,7 @@
 public class ADurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1,
-            false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
index 852b6d4..8737d7b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
@@ -27,7 +27,7 @@
 public class AFloatConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "float", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "float", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
index 46e6987..b282476 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class AInt16ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int16", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int16", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
index a8befc7..3368f41 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class AInt32ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int32", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int32", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
index a3b635d..11242f6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class AInt64ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int64", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int64", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
index 0e6e94d..4bacb74 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class AInt8ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int8", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int8", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
index c64f745..cb90362 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
@@ -26,7 +26,7 @@
 public class ALineConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "line", 1, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "line", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
index ff76339..e9e5353 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ANullConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class ANullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "null", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "null", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -48,7 +48,7 @@
                     private String errorMessage = "This can not be an instance of null";
                     private final byte[] NULL = { 0, 4, 'n', 'u', 'l', 'l' };
                     IBinaryComparator utf8BinaryComparator = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
-                    .createBinaryComparator();
+                            .createBinaryComparator();
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ANULL);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
index 0434a5f..9b6e851 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
@@ -25,8 +25,7 @@
 public class APoint3DConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "point3d", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "point3d", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
index 840f141..f79a403 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
@@ -25,7 +25,7 @@
 public class APointConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "point", 1, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "point", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
index 705ae78..8ad7fa9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
@@ -24,8 +24,7 @@
 public class APolygonConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_POLYGON_TYPE_TAG = ATypeTag.POLYGON.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
index 5f5b3b3..e8a39ca 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
@@ -26,8 +26,7 @@
 public class ARectangleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rectangle", 1,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rectangle", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
index abe1743..d443a5e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
@@ -23,8 +23,7 @@
 public class AStringConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string", 1,
-            false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index c99fc30..fc35bcd 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -27,7 +27,7 @@
 public class ATimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1, false);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1);
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static ByteArrayCharSequenceAccessor charAccessor = new ByteArrayCharSequenceAccessor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
new file mode 100644
index 0000000..c9834ac
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
@@ -0,0 +1,109 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.util.Arrays;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public abstract class AbstractBinaryStringBoolEval implements ICopyEvaluator {
+
+    private DataOutput dout;
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+    private ICopyEvaluator evalLeft;
+    private ICopyEvaluator evalRight;
+    @SuppressWarnings({ "rawtypes" })
+    private ISerializerDeserializer boolSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+    public AbstractBinaryStringBoolEval(DataOutput dout, ICopyEvaluatorFactory evalLeftFactory,
+            ICopyEvaluatorFactory evalRightFactory) throws AlgebricksException {
+        this.dout = dout;
+        this.evalLeft = evalLeftFactory.createEvaluator(array0);
+        this.evalRight = evalRightFactory.createEvaluator(array1);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+        array0.reset();
+        evalLeft.evaluate(tuple);
+        array1.reset();
+        evalRight.evaluate(tuple);
+
+        try {
+            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG && array1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                try {
+                    boolSerde.serialize(ABoolean.TRUE, dout);
+                } catch (HyracksDataException ex) {
+                    Logger.getLogger(AbstractBinaryStringBoolEval.class.getName()).log(Level.SEVERE, null, ex);
+                }
+                return;
+            } else if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG || array1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                boolSerde.serialize(ABoolean.FALSE, dout);
+                return;
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+
+        byte[] b1 = array0.getByteArray();
+        byte[] b2 = array1.getByteArray();
+
+        int lLen = array0.getLength();
+        int rLen = array1.getLength();
+
+        int lStart = array0.getStartOffset();
+        int rStart = array1.getStartOffset();
+        ABoolean res = compute(b1, lLen, lStart, b2, rLen, rStart, array0, array1) ? ABoolean.TRUE : ABoolean.FALSE;
+        try {
+            boolSerde.serialize(res, dout);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    protected abstract boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
+            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) throws AlgebricksException;
+
+    protected String toRegex(AString pattern) {
+        StringBuilder sb = new StringBuilder();
+        String str = pattern.getStringValue();
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            if (c == '\\' && (i < str.length() - 1) && (str.charAt(i + 1) == '_' || str.charAt(i + 1) == '%')) {
+                sb.append(str.charAt(i + 1));
+                ++i;
+            } else if (c == '%') {
+                sb.append(".*");
+            } else if (c == '_') {
+                sb.append(".");
+            } else {
+                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                    sb.append('\\');
+                }
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+
+    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+            '*', '|' };
+    static {
+        Arrays.sort(reservedRegexChars);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
new file mode 100644
index 0000000..96a2876
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -0,0 +1,222 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@SuppressWarnings("serial")
+public abstract class AbstractNumericArithmeticEval extends AbstractScalarFunctionDynamicDescriptor {
+
+    
+    abstract protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException;
+    abstract protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException;
+    
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+                    private DataOutput out = output.getDataOutput();
+                    // one temp. buffer re-used by both children
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
+                    private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
+                    private double[] operandsFloating = new double[args.length];
+                    private long[]   operandsInteger  = new long[args.length];
+                    private int      resultType;
+                    static protected final int typeInt8 = 1;
+                    static protected final int typeInt16 = 2;
+                    static protected final int typeInt32 = 3;
+                    static protected final int typeInt64 = 4;
+                    static protected final int typeFloat = 5;
+                    static protected final int typeDouble = 6;
+                    
+                    
+                    protected AMutableFloat aFloat = new AMutableFloat(0);
+                    protected AMutableDouble aDouble = new AMutableDouble(0);
+                    protected AMutableInt64 aInt64 = new AMutableInt64(0);
+                    protected AMutableInt32 aInt32 = new AMutableInt32(0);
+                    protected AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    protected AMutableInt8 aInt8 = new AMutableInt8((byte) 0);                    
+                    private ATypeTag typeTag;
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+
+                        try {
+                            resultType = 0;
+                            int currentType = 0;
+                            for (int i = 0; i < args.length; i++) {
+                                argOut.reset();
+                                if (i == 0)
+                                    evalLeft.evaluate(tuple);
+                                else
+                                    evalRight.evaluate(tuple);
+                                typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
+                                switch (typeTag) {
+                                    case INT8: {   
+                                        currentType = typeInt8;
+                                        operandsInteger[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                        operandsFloating[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT16: {
+                                        currentType = typeInt16;
+                                        operandsInteger[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                        operandsFloating[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT32: {
+                                        currentType = typeInt32;
+                                        operandsInteger[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                        operandsFloating[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT64: {
+                                        currentType = typeInt64;
+                                        operandsInteger[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                        operandsFloating[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case FLOAT: {
+                                        currentType = typeFloat;
+                                        operandsFloating[i] = AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case DOUBLE: {
+                                        currentType = typeDouble;
+                                        operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case NULL: {
+                                        serde = AqlSerializerDeserializerProvider.INSTANCE
+                                                .getSerializerDeserializer(BuiltinType.ANULL);
+                                        serde.serialize(ANull.NULL, out);
+                                        return;
+                                    }
+                                    default: {
+                                        throw new NotImplementedException(i == 0 ? "Left"
+                                                : "Right"
+                                                        + " Operand of Division can not be "
+                                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
+                                                                .getByteArray()[0]));
+                                    }
+                                }
+                                
+                                if(resultType < currentType) {
+                                    resultType = currentType;
+                                }
+                            }
+                            
+                            long lres = 0;
+                            double dres = 0;
+                            switch(resultType) {
+                                case typeInt8:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.AINT8);
+                                    lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+                                    if(lres > Byte.MAX_VALUE) {
+                                        throw new AlgebricksException("Overflow happened.");
+                                    } 
+                                    if(lres < Byte.MIN_VALUE) {
+                                        throw new AlgebricksException("Underflow happened.");
+                                    }                                     
+                                    aInt8.setValue((byte)lres);
+                                    serde.serialize(aInt8, out);                
+                                    break;                                
+                                case typeInt16:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.AINT16);
+                                    lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+                                    if(lres > Short.MAX_VALUE) {
+                                        throw new AlgebricksException("Overflow happened.");
+                                    } 
+                                    if(lres < Short.MIN_VALUE) {
+                                        throw new AlgebricksException("Underflow happened.");
+                                    }                                    
+                                    aInt16.setValue((short)lres);
+                                    serde.serialize(aInt16, out);                
+                                    break;
+                                case typeInt32:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.AINT32);
+                                    lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
+                                    if(lres > Integer.MAX_VALUE) {
+                                        throw new AlgebricksException("Overflow happened.");
+                                    } 
+                                    if(lres < Integer.MIN_VALUE) {
+                                        throw new AlgebricksException("Underflow happened.");
+                                    }                                      
+                                    aInt32.setValue((int)lres);
+                                    serde.serialize(aInt32, out);                 
+                                    break;
+                                case typeInt64:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.AINT64);
+                                    lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);                                      
+                                    aInt64.setValue(lres);
+                                    serde.serialize(aInt64, out);                   
+                                    break;
+                                case typeFloat:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                    dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
+                                    if(dres > Float.MAX_VALUE) {
+                                        throw new AlgebricksException("Overflow happened.");
+                                    } 
+                                    if(dres < - Float.MAX_VALUE) {
+                                        throw new AlgebricksException("Underflow happened.");
+                                    }                                      
+                                    aFloat.setValue((float)dres);
+                                    serde.serialize(aFloat, out);                   
+                                    break;                                    
+                                case typeDouble:
+                                    serde = AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                    aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
+                                    serde.serialize(aDouble, out);                   
+                                    break;
+                            }                                                   
+                        } catch (HyracksDataException hde) {
+                            throw new AlgebricksException(hde);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
new file mode 100644
index 0000000..a0befd8
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
@@ -0,0 +1,169 @@
+/*
+ * To change this template, choose Tools | Templates
+ * and open the template in the editor.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.regex.Pattern;
+
+/**
+ * @author ilovesoup
+ */
+public abstract class AbstractQuadStringStringEval implements ICopyEvaluator {
+
+    private DataOutput dout;
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array2 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array3 = new ArrayBackedValueStorage();
+    private ICopyEvaluator eval0;
+    private ICopyEvaluator eval1;
+    private ICopyEvaluator eval2;
+    private ICopyEvaluator eval3;
+
+    private AMutableString resultBuffer = new AMutableString("");
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ANULL);
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer strSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+    public AbstractQuadStringStringEval(DataOutput dout, ICopyEvaluatorFactory eval0, ICopyEvaluatorFactory eval1,
+            ICopyEvaluatorFactory eval2, ICopyEvaluatorFactory eval3) throws AlgebricksException {
+        this.dout = dout;
+        this.eval0 = eval0.createEvaluator(array0);
+        this.eval1 = eval1.createEvaluator(array1);
+        this.eval2 = eval2.createEvaluator(array2);
+        this.eval3 = eval3.createEvaluator(array3);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+        array0.reset();
+        eval0.evaluate(tuple);
+        array1.reset();
+        eval1.evaluate(tuple);
+        array2.reset();
+        eval2.evaluate(tuple);
+        array3.reset();
+        eval3.evaluate(tuple);
+
+        try {
+            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                nullSerde.serialize(ANull.NULL, dout);
+                return;
+            } else if (array0.getByteArray()[0] == SER_STRING_TYPE_TAG) {
+                if (array0.getByteArray()[1] == SER_NULL_TYPE_TAG) {
+                    dout.write(array0.getByteArray(), array0.getStartOffset(), array0.getLength());
+                    return;
+                }
+
+            } else {
+                throw new AlgebricksException("Expects String Type.");
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        } catch (IOException e) {
+            throw new AlgebricksException(e.getMessage());
+        }
+
+        byte[] b0 = array0.getByteArray();
+        byte[] b1 = array1.getByteArray();
+        byte[] b2 = array2.getByteArray();
+        byte[] b3 = array3.getByteArray();
+
+        int len0 = array0.getLength();
+        int len1 = array1.getLength();
+        int len2 = array2.getLength();
+        int len3 = array3.getLength();
+
+        int s0 = array0.getStartOffset();
+        int s1 = array1.getStartOffset();
+        int s2 = array2.getStartOffset();
+        int s3 = array3.getStartOffset();
+
+        String res = compute(b0, len0, s0, b1, len1, s1, b2, len2, s2, b3, len3, s3, array0, array1);
+        resultBuffer.setValue(res);
+        try {
+            strSerde.serialize(resultBuffer, dout);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    protected abstract String compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2, int s2,
+            byte[] b3, int l3, int s3, ArrayBackedValueStorage array0, ArrayBackedValueStorage array1)
+            throws AlgebricksException;
+
+    protected String toRegex(AString pattern) {
+        StringBuilder sb = new StringBuilder();
+        String str = pattern.getStringValue();
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            if (c == '\\' && (i < str.length() - 1) && (str.charAt(i + 1) == '_' || str.charAt(i + 1) == '%')) {
+                sb.append(str.charAt(i + 1));
+                ++i;
+            } else if (c == '%') {
+                sb.append(".*");
+            } else if (c == '_') {
+                sb.append(".");
+            } else {
+                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                    sb.append('\\');
+                }
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+
+    protected int toFlag(AString pattern) {
+        String str = pattern.getStringValue();
+        int flag = 0;
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            switch (c) {
+                case 's':
+                    flag |= Pattern.DOTALL;
+                    break;
+                case 'm':
+                    flag |= Pattern.MULTILINE;
+                    break;
+                case 'i':
+                    flag |= Pattern.CASE_INSENSITIVE;
+                    break;
+                case 'x':
+                    flag |= Pattern.COMMENTS;
+                    break;
+            }
+        }
+        return flag;
+    }
+
+    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+            '*', '|' };
+
+    static {
+        Arrays.sort(reservedRegexChars);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
new file mode 100644
index 0000000..e0f8fc9
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
@@ -0,0 +1,141 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.util.Arrays;
+import java.util.regex.Pattern;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public abstract class AbstractTripleStringBoolEval implements ICopyEvaluator {
+
+    private DataOutput dout;
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array2 = new ArrayBackedValueStorage();
+    private ICopyEvaluator eval0;
+    private ICopyEvaluator eval1;
+    private ICopyEvaluator eval2;
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer boolSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+    public AbstractTripleStringBoolEval(DataOutput dout, ICopyEvaluatorFactory eval0, ICopyEvaluatorFactory eval1,
+            ICopyEvaluatorFactory eval2) throws AlgebricksException {
+        this.dout = dout;
+        this.eval0 = eval0.createEvaluator(array0);
+        this.eval1 = eval1.createEvaluator(array1);
+        this.eval2 = eval2.createEvaluator(array2);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+        array0.reset();
+        eval0.evaluate(tuple);
+        array1.reset();
+        eval1.evaluate(tuple);
+        array2.reset();
+        eval2.evaluate(tuple);
+
+        try {
+            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG && array1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                boolSerde.serialize(ABoolean.TRUE, dout);
+                return;
+            } else if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG || array1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                boolSerde.serialize(ABoolean.FALSE, dout);
+                return;
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+
+        byte[] b0 = array0.getByteArray();
+        byte[] b1 = array1.getByteArray();
+        byte[] b2 = array2.getByteArray();
+
+        int len0 = array0.getLength();
+        int len1 = array1.getLength();
+        int len2 = array2.getLength();
+
+        int s0 = array0.getStartOffset();
+        int s1 = array1.getStartOffset();
+        int s2 = array2.getStartOffset();
+
+        ABoolean res = compute(b0, len0, s0, b1, len1, s1, b2, len2, s2, array0, array1) ? ABoolean.TRUE
+                : ABoolean.FALSE;
+        try {
+            boolSerde.serialize(res, dout);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    protected abstract boolean compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2, int s2,
+            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) throws AlgebricksException;
+
+    protected String toRegex(AString pattern) {
+        StringBuilder sb = new StringBuilder();
+        String str = pattern.getStringValue();
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            if (c == '\\' && (i < str.length() - 1) && (str.charAt(i + 1) == '_' || str.charAt(i + 1) == '%')) {
+                sb.append(str.charAt(i + 1));
+                ++i;
+            } else if (c == '%') {
+                sb.append(".*");
+            } else if (c == '_') {
+                sb.append(".");
+            } else {
+                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                    sb.append('\\');
+                }
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+
+    protected int toFlag(AString pattern) {
+        String str = pattern.getStringValue();
+        int flag = 0;
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            switch (c) {
+                case 's':
+                    flag |= Pattern.DOTALL;
+                    break;
+                case 'm':
+                    flag |= Pattern.MULTILINE;
+                    break;
+                case 'i':
+                    flag |= Pattern.CASE_INSENSITIVE;
+                    break;
+                case 'x':
+                    flag |= Pattern.COMMENTS;
+                    break;
+            }
+        }
+        return flag;
+    }
+
+    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+            '*', '|' };
+
+    static {
+        Arrays.sort(reservedRegexChars);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
new file mode 100644
index 0000000..e82d30f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
@@ -0,0 +1,156 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.regex.Pattern;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public abstract class AbstractTripleStringStringEval implements ICopyEvaluator {
+
+    private DataOutput dout;
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+    private ArrayBackedValueStorage array2 = new ArrayBackedValueStorage();
+    private ICopyEvaluator eval0;
+    private ICopyEvaluator eval1;
+    private ICopyEvaluator eval2;
+
+    private AMutableString resultBuffer = new AMutableString("");
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ANULL);
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer strSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+    public AbstractTripleStringStringEval(DataOutput dout, ICopyEvaluatorFactory eval0, ICopyEvaluatorFactory eval1,
+            ICopyEvaluatorFactory eval2) throws AlgebricksException {
+        this.dout = dout;
+        this.eval0 = eval0.createEvaluator(array0);
+        this.eval1 = eval1.createEvaluator(array1);
+        this.eval2 = eval2.createEvaluator(array2);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+        array0.reset();
+        eval0.evaluate(tuple);
+        array1.reset();
+        eval1.evaluate(tuple);
+        array2.reset();
+        eval2.evaluate(tuple);
+
+        try {
+            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+                nullSerde.serialize(ANull.NULL, dout);
+                return;
+            } else if (array0.getByteArray()[0] == SER_STRING_TYPE_TAG) {
+                if (array0.getByteArray()[1] == SER_NULL_TYPE_TAG) {
+                    dout.write(array0.getByteArray(), array0.getStartOffset(), array0.getLength());
+                    return;
+                }
+
+            } else {
+                throw new AlgebricksException("Expects String Type.");
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        } catch (IOException e) {
+            throw new AlgebricksException(e.getMessage());
+        }
+
+        byte[] b0 = array0.getByteArray();
+        byte[] b1 = array1.getByteArray();
+        byte[] b2 = array2.getByteArray();
+
+        int len0 = array0.getLength();
+        int len1 = array1.getLength();
+        int len2 = array2.getLength();
+
+        int s0 = array0.getStartOffset();
+        int s1 = array1.getStartOffset();
+        int s2 = array2.getStartOffset();
+
+        String res = compute(b0, len0, s0, b1, len1, s1, b2, len2, s2, array0, array1);
+        resultBuffer.setValue(res);
+        try {
+            strSerde.serialize(resultBuffer, dout);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    protected abstract String compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2, int s2,
+            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) throws AlgebricksException;
+
+    protected String toRegex(AString pattern) {
+        StringBuilder sb = new StringBuilder();
+        String str = pattern.getStringValue();
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            if (c == '\\' && (i < str.length() - 1) && (str.charAt(i + 1) == '_' || str.charAt(i + 1) == '%')) {
+                sb.append(str.charAt(i + 1));
+                ++i;
+            } else if (c == '%') {
+                sb.append(".*");
+            } else if (c == '_') {
+                sb.append(".");
+            } else {
+                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                    sb.append('\\');
+                }
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+
+    protected int toFlag(AString pattern) {
+        String str = pattern.getStringValue();
+        int flag = 0;
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            switch (c) {
+                case 's':
+                    flag |= Pattern.DOTALL;
+                    break;
+                case 'm':
+                    flag |= Pattern.MULTILINE;
+                    break;
+                case 'i':
+                    flag |= Pattern.CASE_INSENSITIVE;
+                    break;
+                case 'x':
+                    flag |= Pattern.COMMENTS;
+                    break;
+            }
+        }
+        return flag;
+    }
+
+    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+            '*', '|' };
+
+    static {
+        Arrays.sort(reservedRegexChars);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AndDescriptor.java
index 355d30e..3d05651 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -24,7 +24,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
-            "and", FunctionIdentifier.VARARGS, true);
+            "and", FunctionIdentifier.VARARGS);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index a71436c..e4ada73 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -29,7 +29,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "any-collection-member", 1, true);
+            "any-collection-member", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new AnyCollectionMemberDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java
new file mode 100644
index 0000000..240f8c7
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java
@@ -0,0 +1,267 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The most simple implementation of a static hashtable you could imagine.
+ * Intended to work with binary data and be able to map arbitrary key types to
+ * arbitrary value types, given that they have implementations of
+ * IBinaryHashFunction and IBinaryComparator.
+ * 
+ * Uses 2 bytes each to indicate the length of the key and the value.
+ * Uses 8 byte pointers for the linked list (4 bytes frame index, 4 bytes frame offset).
+ * 
+ * This class is NOT thread safe.
+ * 
+ */
+public class BinaryHashMap {
+	// Special value to indicate an empty "bucket" in the header array.
+	private static final long NULL_PTR = -1;
+	private static final int PTR_SIZE = 8;
+	private static final int SLOT_SIZE = 2;
+	private static final int ENTRY_HEADER_SIZE = PTR_SIZE + 2 * SLOT_SIZE;
+	private final IBinaryHashFunction hashFunc;
+	private final IBinaryComparator cmp;
+	private final BinaryEntry returnValue = new BinaryEntry();
+	
+	private final long[] listHeads;
+	private final int frameSize;
+	private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+	private int currFrameIndex;
+	private int nextOff;
+	private int size;
+	
+	// Can be used for key or value.
+	public static class BinaryEntry {
+		public byte[] buf;
+		public int off;
+		public int len;
+		
+		public void set(byte[] buf, int off, int len) {
+			this.buf = buf;
+			this.off = off;
+			this.len = len;
+		}
+		
+		// Inefficient. Just for debugging.
+		@SuppressWarnings("rawtypes")
+		public String print(ISerializerDeserializer serde) throws HyracksDataException {
+			ByteArrayInputStream inStream = new ByteArrayInputStream(buf, off, len);
+            DataInput dataIn = new DataInputStream(inStream);
+            return serde.deserialize(dataIn).toString();
+		}
+	}
+	
+	public BinaryHashMap(int tableSize, int frameSize, IBinaryHashFunction hashFunc, IBinaryComparator cmp) {
+		listHeads = new long[tableSize];
+		this.frameSize = frameSize;		
+		this.hashFunc = hashFunc;
+		this.cmp = cmp;
+		frames.add(ByteBuffer.allocate(frameSize));
+		clear();
+	}
+	
+	/**
+	 * Inserts key, value into the hash map. If key already exists, returns
+	 * existing entry. Otherwise, returns null.
+	 * 
+	 * @param key
+	 * @param value
+	 * @return
+	 */
+	public BinaryEntry put(BinaryEntry key, BinaryEntry value) {
+		return getPutInternal(key, value, true);
+	}
+	
+	/**
+	 * Retrieves value for given key. Returns null if key doesn't exist.
+	 * 
+	 * @param key
+	 * @param value
+	 * @return
+	 */
+	public BinaryEntry get(BinaryEntry key) {
+		return getPutInternal(key, null, false);
+	}
+	
+	private BinaryEntry getPutInternal(BinaryEntry key, BinaryEntry value, boolean put) {
+		int bucket = Math.abs(hashFunc.hash(key.buf, key.off, key.len) % listHeads.length);
+		long headPtr = listHeads[bucket];
+		if (headPtr == NULL_PTR) {
+			// Key definitely doesn't exist yet.
+			if (put) {
+				listHeads[bucket] = appendEntry(key, value);
+			}
+			return null;
+		}
+		// Follow the chain until we found an entry matching the given key.
+		int frameOff;
+		ByteBuffer frame;
+		do {				
+			int frameIndex = getFrameIndex(headPtr);
+			frameOff = getFrameOffset(headPtr);
+			frame = frames.get(frameIndex);
+			int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
+			int entryKeyLen = frame.getShort(frameOff);
+			if (cmp.compare(frame.array(), entryKeyOff, entryKeyLen, key.buf,
+					key.off, key.len) == 0) {
+				// Key found, set values and return.
+				int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
+				int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
+				returnValue.set(frame.array(), entryValOff, entryValLen);
+				return returnValue;
+			}
+			headPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
+		} while (headPtr != NULL_PTR);
+		// We've followed the chain to its end, and didn't find the key.
+		if (put) {
+			// Append the new entry, and set a pointer to it in the last entry we've checked.
+			long newPtr = appendEntry(key, value);
+			frame.putLong(frameOff + 2 * SLOT_SIZE, newPtr);
+		}
+		return null;
+	}
+	
+	public long appendEntry(BinaryEntry key, BinaryEntry value) {
+		ByteBuffer frame = frames.get(currFrameIndex);
+		int requiredSpace = key.len + value.len + ENTRY_HEADER_SIZE;
+		if (nextOff + requiredSpace >= frameSize) {
+			// Entry doesn't fit on frame, allocate a new one.
+			if (requiredSpace > frameSize) {
+				throw new IllegalStateException("Key and value greater than framesize.");
+			}
+			frames.add(ByteBuffer.allocate(frameSize));
+			currFrameIndex++;
+			nextOff = 0;
+			frame = frames.get(currFrameIndex);
+		}
+		writeEntryHeader(frame, nextOff, key.len, value.len, NULL_PTR);
+		System.arraycopy(key.buf, key.off, frame.array(), nextOff + ENTRY_HEADER_SIZE, key.len);
+		System.arraycopy(value.buf, value.off, frame.array(), nextOff + ENTRY_HEADER_SIZE + key.len, value.len);
+		long entryPtr = getEntryPtr(currFrameIndex, nextOff);
+		nextOff += requiredSpace;
+		size++;
+		return entryPtr;
+	}
+	
+	private void writeEntryHeader(ByteBuffer frame, int targetOff, int keyLen, int valLen, long ptr) {
+		frame.putShort(targetOff, (short) keyLen);
+		frame.putShort(targetOff + SLOT_SIZE, (short) valLen);
+		frame.putLong(targetOff + 2 * SLOT_SIZE, ptr);
+	}
+
+	private long getEntryPtr(int frameIndex, int frameOff) {
+        return (((long) frameIndex) << 32) + frameOff;
+    }
+
+    private int getFrameIndex(long ptr) {
+        return (int) ((ptr >> 32) & 0xffffffff);
+    }
+
+    private int getFrameOffset(long ptr) {
+        return (int) (ptr & 0xffffffff);
+    }
+
+	public int size() {
+		return size;
+	}
+
+	public boolean isEmpty() {
+		return size > 0;
+	}
+
+	public void clear() {
+		// Initialize all entries to point to nothing.
+		Arrays.fill(listHeads, NULL_PTR);
+		currFrameIndex = 0;
+		nextOff = 0;
+		size = 0;
+	}
+	
+	public Iterator<Pair<BinaryEntry, BinaryEntry>> iterator() {
+		return new BinaryHashMapIterator();
+	}
+	
+	public class BinaryHashMapIterator implements Iterator<Pair<BinaryEntry, BinaryEntry> > {
+		private final Pair<BinaryEntry, BinaryEntry> val = new Pair<BinaryEntry, BinaryEntry>(new BinaryEntry(), new BinaryEntry());
+		private int listHeadIndex;
+		private ByteBuffer frame;
+		private int frameIndex;
+		private int frameOff;
+		
+		public BinaryHashMapIterator() {
+			listHeadIndex = 0;
+			frame = null;
+			frameIndex = -1;
+			frameOff = -1;
+		}
+		
+		@Override
+		public boolean hasNext() {
+			if (frame != null) {
+				long nextPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
+				if (nextPtr == NULL_PTR) {
+					// End of current list.
+					listHeadIndex++;
+					return nextListHead();
+				} else {
+					// Follow pointer.
+					setValue(nextPtr);
+					return true;
+				}
+			}
+			return nextListHead();
+		}
+
+		private boolean nextListHead() {
+			// Position to first non-null list-head pointer.
+			while(listHeadIndex < listHeads.length && listHeads[listHeadIndex] == NULL_PTR) {
+				listHeadIndex++;
+			}
+			if (listHeadIndex < listHeads.length) {
+				// Positioned to first non-null list head.
+				setValue(listHeads[listHeadIndex]);
+				return true;
+			} else {
+				// No more lists.
+				frame = null;
+				return false;
+			}
+		}
+		
+		private void setValue(long ptr) {
+			frameIndex = getFrameIndex(ptr);
+			frameOff = getFrameOffset(ptr);
+			frame = frames.get(frameIndex);
+			int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
+			int entryKeyLen = frame.getShort(frameOff);
+			int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
+			int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
+			val.first.set(frame.array(), entryKeyOff, entryKeyLen);
+			val.second.set(frame.array(), entryValOff, entryValLen);
+		}
+		
+		@Override
+		public Pair<BinaryEntry, BinaryEntry> next() {
+			return val;
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException("Remove not implemented");
+		}
+	}
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastRecordDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastRecordDescriptor.java
index a9f25f4..aad2c27 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastRecordDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastRecordDescriptor.java
@@ -23,7 +23,7 @@
 public class CastRecordDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     protected static final FunctionIdentifier FID_CAST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "cast-record", 1, true);
+            "cast-record", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CastRecordDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ClosedRecordConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ClosedRecordConstructorDescriptor.java
index 980a6c8..d92aff9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ClosedRecordConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ClosedRecordConstructorDescriptor.java
@@ -12,7 +12,7 @@
 public class ClosedRecordConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     protected static final FunctionIdentifier FID_CLOSED = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "closed-record-constructor", FunctionIdentifier.VARARGS, true);
+            "closed-record-constructor", FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ClosedRecordConstructorDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
new file mode 100644
index 0000000..7e27463
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
@@ -0,0 +1,152 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class CodePointToStringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "codepoint-to-string", 1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new CodePointToStringDescriptor();
+        }
+    };
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
+    private final static byte[] currentUTF8 = new byte[6];
+    private final byte stringTypeTag = ATypeTag.STRING.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            private int codePointToUTF8(int c) {
+                if (c < 0x80) {
+                    currentUTF8[0] = (byte) (c & 0x7F /*mask 7 lsb: 0b1111111 */);
+                    return 1;
+                } else if (c < 0x0800) {
+                    currentUTF8[0] = (byte) (c >> 6 & 0x1F | 0xC0);
+                    currentUTF8[1] = (byte) (c & 0x3F | 0x80);
+                    return 2;
+                } else if (c < 0x010000) {
+                    currentUTF8[0] = (byte) (c >> 12 & 0x0F | 0xE0);
+                    currentUTF8[1] = (byte) (c >> 6 & 0x3F | 0x80);
+                    currentUTF8[2] = (byte) (c & 0x3F | 0x80);
+                    return 3;
+                } else if (c < 0x200000) {
+                    currentUTF8[0] = (byte) (c >> 18 & 0x07 | 0xF0);
+                    currentUTF8[1] = (byte) (c >> 12 & 0x3F | 0x80);
+                    currentUTF8[2] = (byte) (c >> 6 & 0x3F | 0x80);
+                    currentUTF8[3] = (byte) (c & 0x3F | 0x80);
+                    return 4;
+                } else if (c < 0x4000000) {
+                    currentUTF8[0] = (byte) (c >> 24 & 0x03 | 0xF8);
+                    currentUTF8[1] = (byte) (c >> 18 & 0x3F | 0x80);
+                    currentUTF8[2] = (byte) (c >> 12 & 0x3F | 0x80);
+                    currentUTF8[3] = (byte) (c >> 6 & 0x3F | 0x80);
+                    currentUTF8[4] = (byte) (c & 0x3F | 0x80);
+                    return 5;
+                } else if (c < 0x80000000) {
+                    currentUTF8[0] = (byte) (c >> 30 & 0x01 | 0xFC);
+                    currentUTF8[1] = (byte) (c >> 24 & 0x3F | 0x80);
+                    currentUTF8[2] = (byte) (c >> 18 & 0x3F | 0x80);
+                    currentUTF8[3] = (byte) (c >> 12 & 0x3F | 0x80);
+                    currentUTF8[4] = (byte) (c >> 6 & 0x3F | 0x80);
+                    currentUTF8[5] = (byte) (c & 0x3F | 0x80);
+                    return 6;
+                }
+                return 0;
+            }
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ICopyEvaluatorFactory listEvalFactory = args[0];
+                    private ArrayBackedValueStorage outInputList = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalList = listEvalFactory.createEvaluator(outInputList);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            outInputList.reset();
+                            evalList.evaluate(tuple);
+                            byte[] serOrderedList = outInputList.getByteArray();
+                            if (serOrderedList[0] == SER_NULL_TYPE_TAG) {
+                                nullSerde.serialize(ANull.NULL, out);
+                                return;
+                            }
+                            if (serOrderedList[0] != SER_ORDEREDLIST_TYPE_TAG) {
+                                throw new AlgebricksException("Expects an Integer List."
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[0]));
+                            }
+                            int size = AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList);
+                            try {
+                                // calculate length first
+                                int utf_8_len = 0;
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    int codePoint = AInt32SerializerDeserializer.getInt(serOrderedList, itemOffset);
+                                    utf_8_len += codePointToUTF8(codePoint);
+                                }
+                                out.writeByte(stringTypeTag);
+                                StringUtils.writeUTF8Len(utf_8_len, out);
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    int codePoint = AInt32SerializerDeserializer.getInt(serOrderedList, itemOffset);
+                                    utf_8_len = codePointToUTF8(codePoint);
+                                    for (int j = 0; j < utf_8_len; j++) {
+                                        out.writeByte(currentUTF8[j]);
+                                    }
+                                }
+                            } catch (AsterixException ex) {
+                                throw new AlgebricksException(ex);
+                            }
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(e1.getMessage());
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java
new file mode 100644
index 0000000..f103e0f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java
@@ -0,0 +1,101 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+/**
+ *
+ * @author Xiaoyu Ma
+ */
+public class CodepointIterator {
+    public void reset(byte [] buf, int startPos) {
+        this.buf = buf;
+        this.curPos = startPos + 2;
+        this.startPos = startPos;
+        len = UTF8StringPointable.getUTFLength(buf, startPos);        
+    }
+    
+    public int size() { return len; }
+
+    private byte[] buf;
+    private int curPos = 0;
+    private int len = 0;
+    private int startPos = 0;
+
+    public int getCodePoint() {
+        return UTF8ToCodePoint(buf, curPos);
+    }
+
+    public static int UTF8ToCodePoint(byte[] b, int s) {
+        if (b[s] >> 7 == 0) {
+            // 1 byte
+            return b[s];
+        } else if ((b[s] & 0xe0) == 0xc0) { /*
+             * 0xe0 = 0b1110000
+             */
+            // 2 bytes
+            return ((int) (b[s] & 0x1f)) << 6
+                    | /*
+                     * 0x3f = 0b00111111
+                     */ ((int) (b[s + 1] & 0x3f));
+        } else if ((b[s] & 0xf0) == 0xe0) {
+            // 3bytes
+            return ((int) (b[s] & 0xf)) << 12
+                    | ((int) (b[s + 1] & 0x3f)) << 6
+                    | ((int) (b[s + 2] & 0x3f));
+        } else if ((b[s] & 0xf8) == 0xf0) {
+            // 4bytes
+            return ((int) (b[s] & 0x7)) << 18
+                    | ((int) (b[s + 1] & 0x3f)) << 12
+                    | ((int) (b[s + 2] & 0x3f)) << 6
+                    | ((int) (b[s + 3] & 0x3f));
+        } else if ((b[s] & 0xfc) == 0xf8) {
+            // 5bytes
+            return ((int) (b[s] & 0x3)) << 24
+                    | ((int) (b[s + 1] & 0x3f)) << 18
+                    | ((int) (b[s + 2] & 0x3f)) << 12
+                    | ((int) (b[s + 3] & 0x3f)) << 6
+                    | ((int) (b[s + 4] & 0x3f));
+        } else if ((b[s] & 0xfe) == 0xfc) {
+            // 6bytes
+            return ((int) (b[s] & 0x1)) << 30
+                    | ((int) (b[s + 1] & 0x3f)) << 24
+                    | ((int) (b[s + 2] & 0x3f)) << 18
+                    | ((int) (b[s + 3] & 0x3f)) << 12
+                    | ((int) (b[s + 4] & 0x3f)) << 6
+                    | ((int) (b[s + 5] & 0x3f));
+        }
+        return 0;
+    }
+
+    public void next() {
+        int step = UTF8StringPointable.charSize(buf, curPos);
+        if(step + curPos < len + 2 + startPos)
+            curPos += step;
+    }
+
+    public boolean hasNext() {
+        int step = UTF8StringPointable.charSize(buf, curPos);
+        if(step + curPos < len + 2 + startPos)
+            return true;   
+        return false;
+    }
+    
+    public static int compare(CodepointIterator ls, CodepointIterator rs) {
+        CodepointIterator shortString = ls.size() < rs.size() ? ls : rs;
+        
+        while (true) {
+            int c1 = ls.getCodePoint();
+            int c2 = rs.getCodePoint();
+            if (c1 != c2) {
+                return c1 - c2;
+            }
+            if(shortString.hasNext()) {
+                ls.next();
+                rs.next();
+            } else {
+                break;
+            }
+        }
+        return ls.size() - rs.size();
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ContainsDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ContainsDescriptor.java
index ab4fb3a..cf1abc6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ContainsDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/ContainsDescriptor.java
@@ -16,8 +16,7 @@
 public class ContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "contains", 2,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "contains", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ContainsDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
index c8f3447..1e72adf 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -8,20 +7,20 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.GramTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.NGramUTF8StringBinaryTokenizer;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class CountHashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "counthashed-gram-tokens", 3, true);
+            "counthashed-gram-tokens", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CountHashedGramTokensDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
index 4c96ca0..4462098 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -8,21 +7,21 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.HashedUTF8WordTokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.IBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
 
 public class CountHashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "counthashed-word-tokens", 1, true);
+            "counthashed-word-tokens", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CountHashedWordTokensDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
index e8c57c4..dc2c1e1 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
@@ -28,7 +28,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-circle",
-            2, true);
+            2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreateCircleDescriptor();
@@ -68,7 +68,8 @@
                                     APointSerializerDeserializer.getCoordinateOffset(Coordinate.X)),
                                     ADoubleSerializerDeserializer.getDouble(outInput0.getByteArray(),
                                             APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y)));
-                            aCircle.setValue(aPoint, ADoubleSerializerDeserializer.getDouble(outInput1.getByteArray(), 1));
+                            aCircle.setValue(aPoint,
+                                    ADoubleSerializerDeserializer.getDouble(outInput1.getByteArray(), 1));
                             circleSerde.serialize(aCircle, out);
                         } catch (IOException e1) {
                             throw new AlgebricksException(e1);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateLineDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
index a6ca67b..370ed56 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
@@ -27,8 +27,7 @@
 public class CreateLineDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-line", 2,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-line", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreateLineDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
index cd3790c..b1c3564 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
@@ -12,8 +12,7 @@
 public class CreateMBRDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-mbr", 3,
-            true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-mbr", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreateMBRDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index 82d1013..58cd6a8 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -24,8 +24,7 @@
 public class CreatePointDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-point",
-            2, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-point", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreatePointDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
index a095111..f967a69 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
@@ -20,7 +20,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-polygon",
-            FunctionIdentifier.VARARGS, true);
+            FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreatePolygonDescriptor();
@@ -62,7 +62,8 @@
                             outInput.reset();
                             argEvals[i].evaluate(tuple);
                             try {
-                                out.write(outInput.getByteArray(), outInput.getStartOffset() + 1, outInput.getLength() - 1);
+                                out.write(outInput.getByteArray(), outInput.getStartOffset() + 1,
+                                        outInput.getLength() - 1);
                             } catch (IOException e) {
                                 throw new AlgebricksException(e);
                             }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
index d7945fa..44d6881 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
@@ -28,7 +28,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-rectangle", 2, true);
+            "create-rectangle", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new CreateRectangleDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
index 1c1dae3..c261f1f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
@@ -28,7 +28,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "edit-distance-check", 3, true);
+            "edit-distance-check", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EditDistanceCheckDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
index a7246fc..8a7e356 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -16,7 +15,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "edit-distance",
-            2, true);
+            2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EditDistanceDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java
new file mode 100644
index 0000000..f5bfc4f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceListIsFilterable.java
@@ -0,0 +1,128 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+/**
+ * Checks whether a list with an edit distance threshold can be filtered with a lower bounding on the number of common list elements.
+ * This function returns 'true' if the lower bound on the number of common elements is positive, 'false' otherwise.
+ * For example, this function is used during an indexed nested-loop join based on edit distance. We partition the tuples of the probing
+ * dataset into those that are filterable and those that are not. Those that are filterable are forwarded to the index. The others are
+ * are fed into a (non indexed) nested-loop join.
+ */
+public class EditDistanceListIsFilterable extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "edit-distance-list-is-filterable", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new EditDistanceListIsFilterable();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+                return new EditDistanceListIsFilterableEvaluator(args, output);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    private static class EditDistanceListIsFilterableEvaluator implements ICopyEvaluator {
+
+        protected final ArrayBackedValueStorage argBuf = new ArrayBackedValueStorage();
+        protected final IDataOutputProvider output;
+
+        protected final ICopyEvaluator listEval;
+        protected final ICopyEvaluator edThreshEval;
+
+        @SuppressWarnings("unchecked")
+        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+        public EditDistanceListIsFilterableEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
+                throws AlgebricksException {
+            this.output = output;
+            listEval = args[0].createEvaluator(argBuf);
+            edThreshEval = args[1].createEvaluator(argBuf);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+            ATypeTag typeTag = null;
+
+            // Check type and compute string length.
+            argBuf.reset();
+            listEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            int listLen = 0;
+            switch (typeTag) {
+                case UNORDEREDLIST: {
+                    listLen = AUnorderedListSerializerDeserializer.getNumberOfItems(argBuf.getByteArray(), 0);
+                    break;
+                }
+                case ORDEREDLIST: {
+                    listLen = AOrderedListSerializerDeserializer.getNumberOfItems(argBuf.getByteArray(), 0);
+                    break;
+                }
+                default: {
+                    throw new AlgebricksException(
+                            "Expected type 'ORDEREDLIST' or 'UNORDEREDLIST' as first argument. Encountered '"
+                                    + typeTag.toString() + "'.");
+                }
+            }
+
+            // Check type and extract edit-distance threshold.
+            argBuf.reset();
+            edThreshEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            if (!typeTag.equals(ATypeTag.INT32)) {
+                throw new AlgebricksException("Expected type 'INT32' as second argument. Encountered '"
+                        + typeTag.toString() + "'.");
+            }
+            int edThresh = IntegerSerializerDeserializer.getInt(argBuf.getByteArray(), 1);
+
+            // Compute result.
+            int lowerBound = listLen - edThresh;
+            try {
+                if (lowerBound <= 0) {
+                    booleanSerde.serialize(ABoolean.FALSE, output.getDataOutput());
+                } else {
+                    booleanSerde.serialize(ABoolean.TRUE, output.getDataOutput());
+                }
+            } catch (IOException e) {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java
new file mode 100644
index 0000000..443f792
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterable.java
@@ -0,0 +1,149 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+/**
+ * Checks whether a string with an edit distance threshold can be filtered with a lower bounding on number of common grams.
+ * This function returns 'true' if the lower bound on the number of common grams is positive, 'false' otherwise.
+ * For example, this function is used during an indexed nested-loop join based on edit distance. We partition the tuples of the probing
+ * dataset into those that are filterable and those that are not. Those that are filterable are forwarded to the index. The others are
+ * are fed into a (non indexed) nested-loop join.
+ */
+public class EditDistanceStringIsFilterable extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "edit-distance-string-is-filterable", 4);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new EditDistanceStringIsFilterable();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+                return new EditDistanceStringIsFilterableEvaluator(args, output);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    private static class EditDistanceStringIsFilterableEvaluator implements ICopyEvaluator {
+
+        protected final ArrayBackedValueStorage argBuf = new ArrayBackedValueStorage();
+        protected final IDataOutputProvider output;
+
+        protected final ICopyEvaluator stringEval;
+        protected final ICopyEvaluator edThreshEval;
+        protected final ICopyEvaluator gramLenEval;
+        protected final ICopyEvaluator usePrePostEval;
+
+        @SuppressWarnings("unchecked")
+        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+        public EditDistanceStringIsFilterableEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
+                throws AlgebricksException {
+            this.output = output;
+            stringEval = args[0].createEvaluator(argBuf);
+            edThreshEval = args[1].createEvaluator(argBuf);
+            gramLenEval = args[2].createEvaluator(argBuf);
+            usePrePostEval = args[3].createEvaluator(argBuf);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+            ATypeTag typeTag = null;
+
+            // Check type and compute string length.
+            argBuf.reset();
+            stringEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            if (!typeTag.equals(ATypeTag.STRING)) {
+                throw new AlgebricksException("Expected type 'STRING' as first argument. Encountered '"
+                        + typeTag.toString() + "'.");
+            }
+            int utf8Length = UTF8StringPointable.getUTFLength(argBuf.getByteArray(), 1);
+            int pos = 3;
+            int strLen = 0;
+            int end = pos + utf8Length;
+            while (pos < end) {
+                strLen++;
+                pos += UTF8StringPointable.charSize(argBuf.getByteArray(), pos);
+            }
+
+            // Check type and extract edit-distance threshold.
+            argBuf.reset();
+            edThreshEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            if (!typeTag.equals(ATypeTag.INT32)) {
+                throw new AlgebricksException("Expected type 'INT32' as second argument. Encountered '"
+                        + typeTag.toString() + "'.");
+            }
+            int edThresh = IntegerSerializerDeserializer.getInt(argBuf.getByteArray(), 1);
+
+            // Check type and extract gram length.
+            argBuf.reset();
+            gramLenEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            if (!typeTag.equals(ATypeTag.INT32)) {
+                throw new AlgebricksException("Expected type 'INT32' as third argument. Encountered '"
+                        + typeTag.toString() + "'.");
+            }
+            int gramLen = IntegerSerializerDeserializer.getInt(argBuf.getByteArray(), 1);
+
+            // Check type and extract usePrePost flag.
+            argBuf.reset();
+            usePrePostEval.evaluate(tuple);
+            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argBuf.getByteArray()[0]);
+            if (!typeTag.equals(ATypeTag.BOOLEAN)) {
+                throw new AlgebricksException("Expected type 'BOOLEAN' as fourth argument. Encountered '"
+                        + typeTag.toString() + "'.");
+            }
+            boolean usePrePost = BooleanSerializerDeserializer.getBoolean(argBuf.getByteArray(), 1);
+
+            // Compute result.			
+            int numGrams = (usePrePost) ? strLen + gramLen - 1 : strLen - gramLen + 1;
+            int lowerBound = numGrams - edThresh * gramLen;
+            try {
+                if (lowerBound <= 0 || strLen == 0) {
+                    booleanSerde.serialize(ABoolean.FALSE, output.getDataOutput());
+                } else {
+                    booleanSerde.serialize(ABoolean.TRUE, output.getDataOutput());
+                }
+            } catch (IOException e) {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
index 26bfea4..d1f3d25 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EmbedTypeDescriptor.java
@@ -15,14 +15,13 @@
 public class EmbedTypeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "embed-type", 1,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "embed-type", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EmbedTypeDescriptor();
         }
     };
-    
+
     @SuppressWarnings("unused")
     private IAType fieldType;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EndsWithDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EndsWithDescriptor.java
index 19a3be7..0ef6375 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EndsWithDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/EndsWithDescriptor.java
@@ -16,8 +16,7 @@
 public class EndsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ends-with", 2,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ends-with", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EndsWithDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByIndexDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByIndexDescriptor.java
index a0c1ad2..2bb81ad 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByIndexDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByIndexDescriptor.java
@@ -13,13 +13,13 @@
 
     private static final long serialVersionUID = 1L;
     private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-index", 2, true);
+            "field-access-by-index", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessByIndexDescriptor();
         }
     };
-    
+
     private ARecordType recType;
 
     public void reset(ARecordType recType) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByNameDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByNameDescriptor.java
index 2ac483b..b6e376b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByNameDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FieldAccessByNameDescriptor.java
@@ -28,7 +28,7 @@
 
     private static final long serialVersionUID = 1L;
     private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-name", 2, true);
+            "field-access-by-name", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FieldAccessByNameDescriptor();
@@ -55,7 +55,8 @@
         private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
         private final static byte SER_RECORD_TYPE_TAG = ATypeTag.RECORD.serialize();
 
-        public FieldAccessByNameEvalFactory(ICopyEvaluatorFactory recordEvalFactory, ICopyEvaluatorFactory fldNameEvalFactory) {
+        public FieldAccessByNameEvalFactory(ICopyEvaluatorFactory recordEvalFactory,
+                ICopyEvaluatorFactory fldNameEvalFactory) {
             this.recordEvalFactory = recordEvalFactory;
             this.fldNameEvalFactory = fldNameEvalFactory;
         }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
index 623783e..9ad2f2b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/FuzzyEqDescriptor.java
@@ -12,8 +12,7 @@
 public class FuzzyEqDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq", 2,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FuzzyEqDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index f8470b0..e098ef9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -28,8 +28,7 @@
 public class GetItemDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item", 2,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GetItemDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
index 3d1fce6..b01c29c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
@@ -7,20 +7,19 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.GramTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.UTF8NGramTokenFactory;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
 
 public class GramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "gram-tokens",
-            3, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "gram-tokens", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GramTokensDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
index df6bc7b..097df8c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
@@ -7,20 +7,20 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.GramTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.NGramUTF8StringBinaryTokenizer;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class HashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-gram-tokens", 3, true);
+            "hashed-gram-tokens", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new HashedGramTokensDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
index ac756f2..4e44523 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -8,27 +7,27 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.HashedUTF8WordTokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.IBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
 
 public class HashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-word-tokens", 1, true);
+            "hashed-word-tokens", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new HashedWordTokensDescriptor();
         }
     };
-    
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
index abd3ae5..b2ac3ca 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
@@ -21,7 +21,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "inject-failure", 2, true);
+            "inject-failure", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new InjectFailureDescriptor();
@@ -55,7 +55,8 @@
                             // evaluator the failure condition
                             argOut.reset();
                             evals[1].evaluate(tuple);
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(argOut.getByteArray()[0]);
                             if (typeTag == ATypeTag.BOOLEAN) {
                                 boolean argResult = ABooleanSerializerDeserializer.getBoolean(argOut.getByteArray(), 1);
                                 if (argResult)
@@ -65,7 +66,8 @@
                             // evaluate the real evaluator
                             argOut.reset();
                             evals[0].evaluate(tuple);
-                            output.getDataOutput().write(argOut.getByteArray(), argOut.getStartOffset(), argOut.getLength());
+                            output.getDataOutput().write(argOut.getByteArray(), argOut.getStartOffset(),
+                                    argOut.getLength());
                         } catch (IOException e) {
                             throw new AlgebricksException(e);
                         }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/IsNullDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/IsNullDescriptor.java
index 5e3b7ab..813fc41 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/IsNullDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/IsNullDescriptor.java
@@ -23,7 +23,7 @@
     private static final long serialVersionUID = 1L;
 
     public final static FunctionIdentifier FID = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
-            "is-null", 1, true);
+            "is-null", 1);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LenDescriptor.java
index 0e15a18..e09035e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -28,7 +28,7 @@
 public class LenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1);
     private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
     private final static byte SER_UNORDEREDLIST_TYPE_TAG = ATypeTag.UNORDEREDLIST.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LikeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LikeDescriptor.java
index 1b43441..a3bd5bd 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LikeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/LikeDescriptor.java
@@ -39,7 +39,7 @@
 public class LikeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -100,8 +100,8 @@
                                 first = false;
                                 newPattern = true;
                             } else {
-                                int c = strComp.compare(array0.getByteArray(), array0.getStartOffset(), array0.getLength(),
-                                        lastPattern.getByteArray(), 0, lastPattern.size());
+                                int c = strComp.compare(array0.getByteArray(), array0.getStartOffset(),
+                                        array0.getLength(), lastPattern.getByteArray(), 0, lastPattern.size());
                                 if (c != 0) {
                                     newPattern = true;
                                 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NotDescriptor.java
index abb4e25..a3f78e4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -26,7 +26,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
-            "not", 1, true);
+            "not", 1);
 
     private final static byte SER_BOOLEAN_TYPE_TAG = ATypeTag.BOOLEAN.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
new file mode 100644
index 0000000..662e777
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
@@ -0,0 +1,145 @@
+/*
+ * Numeric function Absolute value
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class NumericAbsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-abs", 1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericAbsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue((val <= 0) ? (byte) (0 - val) : val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue((val <= 0) ? (short) (0 - val) : val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue((val <= 0) ? (0 - val) : val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue((val <= 0L) ? (0L - val) : val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                aFloat.setValue((val <= 0.0f) ? 0.0f - val : val);
+                                serde.serialize(aFloat, out);
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                aDouble.setValue((val <= 0.0D) ? 0.0D - val : val);
+                                serde.serialize(aDouble, out);
+                            } else {
+                                throw new NotImplementedException("Numeric Abs is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index baa20cd..c2791b9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -1,44 +1,16 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-import java.io.DataOutput;
-
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AMutableDouble;
-import edu.uci.ics.asterix.om.base.AMutableFloat;
-import edu.uci.ics.asterix.om.base.AMutableInt16;
-import edu.uci.ics.asterix.om.base.AMutableInt32;
-import edu.uci.ics.asterix.om.base.AMutableInt64;
-import edu.uci.ics.asterix.om.base.AMutableInt8;
-import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class NumericAddDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NumericAddDescriptor extends AbstractNumericArithmeticEval {
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
-            "numeric-add", 2, true);
+            "numeric-add", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericAddDescriptor();
@@ -51,130 +23,18 @@
     }
 
     @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
-
-                return new ICopyEvaluator() {
-                    private DataOutput out = output.getDataOutput();
-                    // one temp. buffer re-used by both children
-                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
-                    private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
-                    private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
-                    private double[] operands = new double[args.length];
-                    private boolean metInt8 = false, metInt16 = false, metInt32 = false, metInt64 = false,
-                            metFloat = false, metDouble = false;
-                    private ATypeTag typeTag;
-                    private AMutableDouble aDouble = new AMutableDouble(0);
-                    private AMutableFloat aFloat = new AMutableFloat(0);
-                    private AMutableInt64 aInt64 = new AMutableInt64(0);
-                    private AMutableInt32 aInt32 = new AMutableInt32(0);
-                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
-                    @SuppressWarnings("rawtypes")
-                    private ISerializerDeserializer serde;
-
-                    @SuppressWarnings("unchecked")
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-
-                        try {
-                            for (int i = 0; i < args.length; i++) {
-                                argOut.reset();
-                                if (i == 0)
-                                    evalLeft.evaluate(tuple);
-                                else
-                                    evalRight.evaluate(tuple);
-                                typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
-                                switch (typeTag) {
-                                    case INT8: {
-                                        metInt8 = true;
-                                        operands[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT16: {
-                                        metInt16 = true;
-                                        operands[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT32: {
-                                        metInt32 = true;
-                                        operands[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT64: {
-                                        metInt64 = true;
-                                        operands[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case FLOAT: {
-                                        metFloat = true;
-                                        operands[i] = AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case DOUBLE: {
-                                        metDouble = true;
-                                        operands[i] = ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case NULL: {
-                                        serde = AqlSerializerDeserializerProvider.INSTANCE
-                                                .getSerializerDeserializer(BuiltinType.ANULL);
-                                        serde.serialize(ANull.NULL, out);
-                                        return;
-                                    }
-                                    default: {
-                                        throw new NotImplementedException(i == 0 ? "Left"
-                                                : "Right"
-                                                        + " Operand of Addition can not be "
-                                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
-                                                                .getByteArray()[0]));
-                                    }
-                                }
-                            }
-
-                            if (metDouble) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
-                                aDouble.setValue(operands[0] + operands[1]);
-                                serde.serialize(aDouble, out);
-                            } else if (metFloat) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
-                                aFloat.setValue((float) (operands[0] + operands[1]));
-                                serde.serialize(aFloat, out);
-                            } else if (metInt64) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT64);
-                                aInt64.setValue((long) (operands[0] + operands[1]));
-                                serde.serialize(aInt64, out);
-                            } else if (metInt32) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT32);
-                                aInt32.setValue((int) (operands[0] + operands[1]));
-                                serde.serialize(aInt32, out);
-                            } else if (metInt16) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT16);
-                                aInt16.setValue((short) (operands[0] + operands[1]));
-                                serde.serialize(aInt16, out);
-                            } else if (metInt8) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT8);
-                                aInt8.setValue((byte) (operands[0] + operands[1]));
-                                serde.serialize(aInt8, out);
-                            }
-
-                        } catch (HyracksDataException hde) {
-                            throw new AlgebricksException(hde);
-                        }
-                    }
-                };
-            }
-        };
+    protected long evaluateInteger(long x, long y) throws HyracksDataException {
+        long z = x + y;
+        if (x > 0) {
+            if (y > 0 && z < 0)
+                throw new ArithmeticException("Overflow adding " + x + " + " + y);
+        } else if (y < 0 && z > 0)
+            throw new ArithmeticException("Overflow adding " + x + " + " + y);
+        return z;
     }
 
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+        return lhs + rhs;
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
new file mode 100644
index 0000000..2029426
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -0,0 +1,147 @@
+/*
+ * Numeric function ceiling
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class NumericCeilingDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-ceiling", 1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericCeilingDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue(val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue(val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue(val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue(val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                aFloat.setValue((float) Math.ceil(val));
+                                serde.serialize(aFloat, out);
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                aDouble.setValue(Math.ceil(val));
+                                serde.serialize(aDouble, out);
+                            } else {
+                                throw new NotImplementedException("Numeric Ceiling is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index b226ad7..87a4257 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -1,44 +1,16 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-import java.io.DataOutput;
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AMutableDouble;
-import edu.uci.ics.asterix.om.base.AMutableFloat;
-import edu.uci.ics.asterix.om.base.AMutableInt16;
-import edu.uci.ics.asterix.om.base.AMutableInt32;
-import edu.uci.ics.asterix.om.base.AMutableInt64;
-import edu.uci.ics.asterix.om.base.AMutableInt8;
-import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class NumericDivideDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NumericDivideDescriptor extends AbstractNumericArithmeticEval {
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-divide",
-            2, true);
+            2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericDivideDescriptor();
@@ -51,130 +23,14 @@
     }
 
     @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
-
-                return new ICopyEvaluator() {
-                    private DataOutput out = output.getDataOutput();
-                    // one temp. buffer re-used by both children
-                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
-                    private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
-                    private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
-                    private double[] operands = new double[args.length];
-                    private boolean metInt8 = false, metInt16 = false, metInt32 = false, metInt64 = false,
-                            metFloat = false, metDouble = false;
-                    private ATypeTag typeTag;
-                    private AMutableDouble aDouble = new AMutableDouble(0);
-                    private AMutableFloat aFloat = new AMutableFloat(0);
-                    private AMutableInt64 aInt64 = new AMutableInt64(0);
-                    private AMutableInt32 aInt32 = new AMutableInt32(0);
-                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
-                    @SuppressWarnings("rawtypes")
-                    private ISerializerDeserializer serde;
-
-                    @SuppressWarnings("unchecked")
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-
-                        try {
-                            for (int i = 0; i < args.length; i++) {
-                                argOut.reset();
-                                if (i == 0)
-                                    evalLeft.evaluate(tuple);
-                                else
-                                    evalRight.evaluate(tuple);
-                                typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
-                                switch (typeTag) {
-                                    case INT8: {
-                                        metInt8 = true;
-                                        operands[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT16: {
-                                        metInt16 = true;
-                                        operands[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT32: {
-                                        metInt32 = true;
-                                        operands[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT64: {
-                                        metInt64 = true;
-                                        operands[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case FLOAT: {
-                                        metFloat = true;
-                                        operands[i] = AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case DOUBLE: {
-                                        metDouble = true;
-                                        operands[i] = ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case NULL: {
-                                        serde = AqlSerializerDeserializerProvider.INSTANCE
-                                                .getSerializerDeserializer(BuiltinType.ANULL);
-                                        serde.serialize(ANull.NULL, out);
-                                        return;
-                                    }
-                                    default: {
-                                        throw new NotImplementedException(i == 0 ? "Left"
-                                                : "Right"
-                                                        + " Operand of Division can not be "
-                                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
-                                                                .getByteArray()[0]));
-                                    }
-                                }
-                            }
-
-                            if (metDouble) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
-                                aDouble.setValue(operands[0] / operands[1]);
-                                serde.serialize(aDouble, out);
-                            } else if (metFloat) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
-                                aFloat.setValue((float) (operands[0] / operands[1]));
-                                serde.serialize(aFloat, out);
-                            } else if (metInt64) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT64);
-                                aInt64.setValue((long) (operands[0] / operands[1]));
-                                serde.serialize(aInt64, out);
-                            } else if (metInt32) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT32);
-                                aInt32.setValue((int) (operands[0] / operands[1]));
-                                serde.serialize(aInt32, out);
-                            } else if (metInt16) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT16);
-                                aInt16.setValue((short) (operands[0] / operands[1]));
-                                serde.serialize(aInt16, out);
-                            } else if (metInt8) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT8);
-                                aInt8.setValue((byte) (operands[0] / operands[1]));
-                                serde.serialize(aInt8, out);
-                            }
-
-                        } catch (HyracksDataException hde) {
-                            throw new AlgebricksException(hde);
-                        }
-                    }
-                };
-            }
-        };
+    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+        if (rhs == 0)
+            throw new HyracksDataException("Divide by Zero.");
+        return lhs / rhs;
     }
 
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) {
+        return lhs / rhs;
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
new file mode 100644
index 0000000..d30ebb5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
@@ -0,0 +1,146 @@
+/*
+ * Numeric function Floor
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class NumericFloorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-floor",
+            1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericFloorDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue(val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue(val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue(val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue(val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                aFloat.setValue((float) Math.floor(val));
+                                serde.serialize(aFloat, out);
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                aDouble.setValue(Math.floor(val));
+                                serde.serialize(aDouble, out);
+                            } else {
+                                throw new NotImplementedException("Numeric Floor is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
new file mode 100644
index 0000000..c92d75f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
@@ -0,0 +1,195 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * @author kisskys
+ */
+public class NumericModuloDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-mod", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericModuloDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+                    private DataOutput out = output.getDataOutput();
+                    // one temp. buffer re-used by both children
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
+                    private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
+                    private double[] operands = new double[args.length];
+                    private boolean metInt8 = false, metInt16 = false, metInt32 = false, metInt64 = false,
+                            metFloat = false, metDouble = false;
+                    private ATypeTag typeTag;
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+
+                        try {
+                            for (int i = 0; i < args.length; i++) {
+                                argOut.reset();
+                                if (i == 0)
+                                    evalLeft.evaluate(tuple);
+                                else
+                                    evalRight.evaluate(tuple);
+                                typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
+                                switch (typeTag) {
+                                    case INT8: {
+                                        metInt8 = true;
+                                        operands[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT16: {
+                                        metInt16 = true;
+                                        operands[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT32: {
+                                        metInt32 = true;
+                                        operands[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case INT64: {
+                                        metInt64 = true;
+                                        operands[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case FLOAT: {
+                                        metFloat = true;
+                                        operands[i] = AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case DOUBLE: {
+                                        metDouble = true;
+                                        operands[i] = ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                        break;
+                                    }
+                                    case NULL: {
+                                        serde = AqlSerializerDeserializerProvider.INSTANCE
+                                                .getSerializerDeserializer(BuiltinType.ANULL);
+                                        serde.serialize(ANull.NULL, out);
+                                        return;
+                                    }
+                                    default: {
+                                        throw new NotImplementedException(i == 0 ? "Left" : "Right"
+                                                + " Operand of Modulo can not be "
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
+                                                        .getByteArray()[0]));
+                                    }
+                                }
+                            }
+
+                            if (metDouble) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                aDouble.setValue(operands[0] % operands[1]);
+                                serde.serialize(aDouble, out);
+                            } else if (metFloat) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                aFloat.setValue((float) (operands[0] % operands[1]));
+                                serde.serialize(aFloat, out);
+                            } else if (metInt64) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                aInt64.setValue((long) (operands[0] % operands[1]));
+                                serde.serialize(aInt64, out);
+                            } else if (metInt32) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                aInt32.setValue((int) (operands[0] % operands[1]));
+                                serde.serialize(aInt32, out);
+                            } else if (metInt16) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                aInt16.setValue((short) (operands[0] % operands[1]));
+                                serde.serialize(aInt16, out);
+                            } else if (metInt8) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                aInt8.setValue((byte) (operands[0] % operands[1]));
+                                serde.serialize(aInt8, out);
+                            }
+
+                        } catch (HyracksDataException hde) {
+                            throw new AlgebricksException(hde);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index aa5eaaa..606f2ec 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -1,44 +1,16 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-import java.io.DataOutput;
-
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AMutableDouble;
-import edu.uci.ics.asterix.om.base.AMutableFloat;
-import edu.uci.ics.asterix.om.base.AMutableInt16;
-import edu.uci.ics.asterix.om.base.AMutableInt32;
-import edu.uci.ics.asterix.om.base.AMutableInt64;
-import edu.uci.ics.asterix.om.base.AMutableInt8;
-import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class NumericMultiplyDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NumericMultiplyDescriptor extends AbstractNumericArithmeticEval {
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-multiply", 2, true);
+            "numeric-multiply", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericMultiplyDescriptor();
@@ -51,129 +23,26 @@
     }
 
     @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
+    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+        int signLhs = lhs > 0 ? 1 : (lhs < 0 ? -1 : 0);
+        int signRhs = rhs > 0 ? 1 : (rhs < 0 ? -1 : 0);
+        long maximum = signLhs == signRhs ? Long.MAX_VALUE : Long.MIN_VALUE;
 
-            @Override
-            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+        if (lhs != 0 && (rhs > 0 && rhs > maximum / lhs || rhs < 0 && rhs < maximum / lhs))
+            throw new HyracksDataException("Overflow Happened.");
 
-                return new ICopyEvaluator() {
-                    private DataOutput out = output.getDataOutput();
-                    // one temp. buffer re-used by both children
-                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
-                    private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
-                    private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
-                    private double[] operands = new double[args.length];
-                    private boolean metInt8 = false, metInt16 = false, metInt32 = false, metInt64 = false,
-                            metFloat = false, metDouble = false;
-                    private ATypeTag typeTag;
-                    private AMutableDouble aDouble = new AMutableDouble(0);
-                    private AMutableFloat aFloat = new AMutableFloat(0);
-                    private AMutableInt64 aInt64 = new AMutableInt64(0);
-                    private AMutableInt32 aInt32 = new AMutableInt32(0);
-                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
-                    @SuppressWarnings("rawtypes")
-                    private ISerializerDeserializer serde;
-
-                    @SuppressWarnings("unchecked")
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-
-                        try {
-                            for (int i = 0; i < args.length; i++) {
-                                argOut.reset();
-                                if (i == 0)
-                                    evalLeft.evaluate(tuple);
-                                else
-                                    evalRight.evaluate(tuple);
-                                typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
-                                switch (typeTag) {
-                                    case INT8: {
-                                        metInt8 = true;
-                                        operands[i] = AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT16: {
-                                        metInt16 = true;
-                                        operands[i] = AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT32: {
-                                        metInt32 = true;
-                                        operands[i] = AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case INT64: {
-                                        metInt64 = true;
-                                        operands[i] = AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case FLOAT: {
-                                        metFloat = true;
-                                        operands[i] = AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case DOUBLE: {
-                                        metDouble = true;
-                                        operands[i] = ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
-                                        break;
-                                    }
-                                    case NULL: {
-                                        serde = AqlSerializerDeserializerProvider.INSTANCE
-                                                .getSerializerDeserializer(BuiltinType.ANULL);
-                                        serde.serialize(ANull.NULL, out);
-                                        return;
-                                    }
-                                    default: {
-                                        throw new NotImplementedException(i == 0 ? "Left"
-                                                : "Right"
-                                                        + " Operand of Multiplication can not be "
-                                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
-                                                                .getByteArray()[0]));
-                                    }
-                                }
-                            }
-                            if (metDouble) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
-                                aDouble.setValue(operands[0] * operands[1]);
-                                serde.serialize(aDouble, out);
-                            } else if (metFloat) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
-                                aFloat.setValue((float) (operands[0] * operands[1]));
-                                serde.serialize(aFloat, out);
-                            } else if (metInt64) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT64);
-                                aInt64.setValue((long) (operands[0] * operands[1]));
-                                serde.serialize(aInt64, out);
-                            } else if (metInt32) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT32);
-                                aInt32.setValue((int) (operands[0] * operands[1]));
-                                serde.serialize(aInt32, out);
-                            } else if (metInt16) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT16);
-                                aInt16.setValue((short) (operands[0] * operands[1]));
-                                serde.serialize(aInt16, out);
-                            } else if (metInt8) {
-                                serde = AqlSerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT8);
-                                aInt8.setValue((byte) (operands[0] * operands[1]));
-                                serde.serialize(aInt8, out);
-                            }
-
-                        } catch (HyracksDataException hde) {
-                            throw new AlgebricksException(hde);
-                        }
-                    }
-                };
-            }
-        };
+        return lhs * rhs;
     }
 
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+        int signLhs = lhs > 0 ? 1 : (lhs < 0 ? -1 : 0);
+        int signRhs = rhs > 0 ? 1 : (rhs < 0 ? -1 : 0);
+        double maximum = signLhs == signRhs ? Double.MAX_VALUE : -Double.MAX_VALUE;
+
+        if (lhs != 0 && (rhs > 0 && rhs > maximum / lhs || rhs < 0 && rhs < maximum / lhs))
+            throw new HyracksDataException("Overflow Happened.");
+
+        return lhs * rhs;
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
new file mode 100644
index 0000000..d43cdd0
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
@@ -0,0 +1,148 @@
+/*
+ * Numeric function Round
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableFloat;
+import edu.uci.ics.asterix.om.base.AMutableInt16;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class NumericRoundDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-round",
+            1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericRoundDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue(val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue(val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue(val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue(val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                val = Math.round(val);
+                                aFloat.setValue(val);
+                                serde.serialize(aFloat, out);
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                val = Math.round(val);
+                                aDouble.setValue(val);
+                                serde.serialize(aDouble, out);
+                            } else {
+                                throw new NotImplementedException("Numeric Round is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
new file mode 100644
index 0000000..f43567c
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
@@ -0,0 +1,174 @@
+/*
+ * Numeric function Round half to even
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.*;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.*;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.math.BigDecimal;
+
+public class NumericRoundHalfToEven2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-round-half-to-even2", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericRoundHalfToEven2Descriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private ICopyEvaluator precision = args[1].createEvaluator(argOut);
+
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    private int getPrecision(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        precision.evaluate(tuple);
+
+                        if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                            return (int) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                        } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                            return (int) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                        } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                            return (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                        } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                            return (int) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                        } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                            return (int) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                        } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                            return (int) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                        }
+
+                        return 0;
+                    }
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue(val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue(val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue(val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue(val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                if (Float.isNaN(val) || Float.isInfinite(val) || val == -0.0F || val == 0.0F) {
+                                    aFloat.setValue(val);
+                                    serde.serialize(aFloat, out);
+                                } else {
+                                    BigDecimal r = new BigDecimal(Float.toString(val));
+                                    aFloat.setValue(r.setScale(getPrecision(tuple), BigDecimal.ROUND_HALF_EVEN)
+                                            .floatValue());
+                                    serde.serialize(aFloat, out);
+                                }
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                if (Double.isNaN(val) || Double.isInfinite(val) || val == -0.0D || val == 0.0D) {
+                                    aDouble.setValue(val);
+                                    serde.serialize(aDouble, out);
+                                } else {
+                                    BigDecimal r = new BigDecimal(Double.toString(val));
+                                    aDouble.setValue(r.setScale(getPrecision(tuple), BigDecimal.ROUND_HALF_EVEN)
+                                            .doubleValue());
+                                    serde.serialize(aDouble, out);
+                                }
+                            } else {
+                                throw new NotImplementedException("Numeric Round Half to Even is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
new file mode 100644
index 0000000..9499bb3
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * Numeric function Round half to even
+ * Author : Xiaoyu Ma@UC Irvine
+ * 01/30/2012
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.*;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.*;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+
+public class NumericRoundHalfToEvenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-round-half-to-even", 1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericRoundHalfToEvenDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+                    private byte serNullTypeTag = ATypeTag.NULL.serialize();
+                    private byte serInt8TypeTag = ATypeTag.INT8.serialize();
+                    private byte serInt16TypeTag = ATypeTag.INT16.serialize();
+                    private byte serInt32TypeTag = ATypeTag.INT32.serialize();
+                    private byte serInt64TypeTag = ATypeTag.INT64.serialize();
+                    private byte serFloatTypeTag = ATypeTag.FLOAT.serialize();
+                    private byte serDoubleTypeTag = ATypeTag.DOUBLE.serialize();
+
+                    private AMutableDouble aDouble = new AMutableDouble(0);
+                    private AMutableFloat aFloat = new AMutableFloat(0);
+                    private AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    @SuppressWarnings("rawtypes")
+                    private ISerializerDeserializer serde;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        eval.evaluate(tuple);
+                        try {
+                            if (argOut.getByteArray()[0] == serNullTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ANULL);
+                                serde.serialize(ANull.NULL, out);
+                                return;
+                            } else if (argOut.getByteArray()[0] == serInt8TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                                byte val = (byte) AInt8SerializerDeserializer.getByte(argOut.getByteArray(), 1);
+                                aInt8.setValue(val);
+                                serde.serialize(aInt8, out);
+                            } else if (argOut.getByteArray()[0] == serInt16TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                                short val = (short) AInt16SerializerDeserializer.getShort(argOut.getByteArray(), 1);
+                                aInt16.setValue(val);
+                                serde.serialize(aInt16, out);
+                            } else if (argOut.getByteArray()[0] == serInt32TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                                int val = (int) AInt32SerializerDeserializer.getInt(argOut.getByteArray(), 1);
+                                aInt32.setValue(val);
+                                serde.serialize(aInt32, out);
+                            } else if (argOut.getByteArray()[0] == serInt64TypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                                long val = (long) AInt64SerializerDeserializer.getLong(argOut.getByteArray(), 1);
+                                aInt64.setValue(val);
+                                serde.serialize(aInt64, out);
+                            } else if (argOut.getByteArray()[0] == serFloatTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                                float val = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), 1);
+                                aFloat.setValue((float) Math.rint(val));
+                                serde.serialize(aFloat, out);
+                            } else if (argOut.getByteArray()[0] == serDoubleTypeTag) {
+                                serde = AqlSerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
+                                double val = (double) ADoubleSerializerDeserializer.getDouble(argOut.getByteArray(), 1);
+                                aDouble.setValue(Math.rint(val));
+                                serde.serialize(aDouble, out);
+                            } else {
+                                throw new NotImplementedException("Numeric Round Half to Even is not implemented for "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]));
+                            }
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
index 0b53f18..9abbf47 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
@@ -38,7 +38,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-subtract", 2, true);
+            "numeric-subtract", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericSubtractDescriptor();
@@ -127,11 +127,10 @@
                                         return;
                                     }
                                     default: {
-                                        throw new NotImplementedException(i == 0 ? "Left"
-                                                : "Right"
-                                                        + " Operand of Substraction can not be "
-                                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
-                                                                .getByteArray()[0]));
+                                        throw new NotImplementedException(i == 0 ? "Left" : "Right"
+                                                + " Operand of Substraction can not be "
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
+                                                        .getByteArray()[0]));
                                     }
                                 }
                             }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
index 7998618..9e3b188 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
@@ -38,7 +38,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-unary-minus", 1, true);
+            "numeric-unary-minus", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericUnaryMinusDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OpenRecordConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OpenRecordConstructorDescriptor.java
index f3681f2..37d6ce0 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OpenRecordConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OpenRecordConstructorDescriptor.java
@@ -21,7 +21,7 @@
 public class OpenRecordConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     protected static final FunctionIdentifier FID_OPEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "open-record-constructor", FunctionIdentifier.VARARGS, true);
+            "open-record-constructor", FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OpenRecordConstructorDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrDescriptor.java
index 8349c97..a54ec54 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrDescriptor.java
@@ -24,7 +24,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, "or",
-            FunctionIdentifier.VARARGS, true);
+            FunctionIdentifier.VARARGS);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
index b9e341d..520b386 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/OrderedListConstructorDescriptor.java
@@ -22,7 +22,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "ordered-list-constructor", FunctionIdentifier.VARARGS, true);
+            "ordered-list-constructor", FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OrderedListConstructorDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
index 57b7acd..25cbf74 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
@@ -27,8 +27,7 @@
 public class PrefixLenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "prefix-len", 3,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "prefix-len", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new PrefixLenDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
index 2612583..927ade5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
@@ -27,7 +27,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "prefix-len-jaccard", 2, true);
+            "prefix-len-jaccard", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new PrefixLenJaccardDescriptor();
@@ -69,8 +69,8 @@
                         // similarity threshold
                         inputVal.reset();
                         evalThreshold.evaluate(tuple);
-                        float similarityThreshold = (float) AFloatSerializerDeserializer.getFloat(inputVal.getByteArray(),
-                                1);
+                        float similarityThreshold = (float) AFloatSerializerDeserializer.getFloat(
+                                inputVal.getByteArray(), 1);
 
                         if (similarityThreshold != similarityThresholdCache || similarityFilters == null) {
                             similarityFilters = new SimilarityFiltersJaccard(similarityThreshold);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/RegExpDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/RegExpDescriptor.java
index 1815085..01b00a0 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/RegExpDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/RegExpDescriptor.java
@@ -38,8 +38,7 @@
 public class RegExpDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -100,8 +99,8 @@
                                 first = false;
                                 newPattern = true;
                             } else {
-                                int c = strComp.compare(array0.getByteArray(), array0.getStartOffset(), array0.getLength(),
-                                        lastPattern.getByteArray(), 0, lastPattern.size());
+                                int c = strComp.compare(array0.getByteArray(), array0.getStartOffset(),
+                                        array0.getLength(), lastPattern.getByteArray(), 0, lastPattern.size());
                                 if (c != 0) {
                                     newPattern = true;
                                 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index d3218ff..ec21b96 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -35,8 +35,7 @@
 public class SimilarityDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "similarity", 7,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "similarity", 7);
 
     private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
     private final static byte SER_UNORDEREDLIST_TYPE_TAG = ATypeTag.UNORDEREDLIST.serialize();
@@ -122,8 +121,8 @@
 
                             int lengthTokens1;
                             if (serList[0] == SER_ORDEREDLIST_TYPE_TAG) {
-                                lengthTokens1 = AOrderedListSerializerDeserializer
-                                        .getNumberOfItems(inputVal.getByteArray());
+                                lengthTokens1 = AOrderedListSerializerDeserializer.getNumberOfItems(inputVal
+                                        .getByteArray());
                                 // read tokens
                                 for (i = 0; i < lengthTokens1; i++) {
                                     int itemOffset;
@@ -166,8 +165,8 @@
 
                             int lengthTokens2;
                             if (serList[0] == SER_ORDEREDLIST_TYPE_TAG) {
-                                lengthTokens2 = AOrderedListSerializerDeserializer
-                                        .getNumberOfItems(inputVal.getByteArray());
+                                lengthTokens2 = AOrderedListSerializerDeserializer.getNumberOfItems(inputVal
+                                        .getByteArray());
                                 // read tokens
                                 for (i = 0; i < lengthTokens2; i++) {
                                     int itemOffset;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
index 296916d..4881e2a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
@@ -1,36 +1,21 @@
 package edu.uci.ics.asterix.runtime.evaluators.functions;
 
-import java.io.IOException;
-
-import edu.uci.ics.asterix.builders.IAOrderedListBuilder;
-import edu.uci.ics.asterix.builders.OrderedListBuilder;
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ABoolean;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.common.SimilarityJaccardEvaluator;
+import edu.uci.ics.asterix.runtime.evaluators.common.SimilarityJaccardCheckEvaluator;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-//assumes that both arguments are sorted by the same ordering
 
 public class SimilarityJaccardCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard-check", 3, true);
+            "similarity-jaccard-check", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SimilarityJaccardCheckDescriptor();
@@ -53,66 +38,4 @@
     public FunctionIdentifier getIdentifier() {
         return FID;
     }
-
-    private static class SimilarityJaccardCheckEvaluator extends SimilarityJaccardEvaluator {
-
-        private final ICopyEvaluator jaccThreshEval;
-        private float jaccThresh = -1f;
-
-        private IAOrderedListBuilder listBuilder;
-        private ArrayBackedValueStorage inputVal;
-        @SuppressWarnings("unchecked")
-        private final ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-        private final AOrderedListType listType = new AOrderedListType(BuiltinType.ANY, "list");
-
-        public SimilarityJaccardCheckEvaluator(ICopyEvaluatorFactory[] args, IDataOutputProvider output)
-                throws AlgebricksException {
-            super(args, output);
-            jaccThreshEval = args[2].createEvaluator(argOut);
-            listBuilder = new OrderedListBuilder();
-            inputVal = new ArrayBackedValueStorage();
-        }
-
-        @Override
-        protected void runArgEvals(IFrameTupleReference tuple) throws AlgebricksException {
-            super.runArgEvals(tuple);
-            int jaccThreshStart = argOut.getLength();
-            jaccThreshEval.evaluate(tuple);
-            jaccThresh = (float) AFloatSerializerDeserializer.getFloat(argOut.getByteArray(), jaccThreshStart
-                    + typeIndicatorSize);
-        }
-
-        @Override
-        protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
-                throws AlgebricksException {
-            firstListIter.reset(bytes, firstStart);
-            secondListIter.reset(bytes, secondStart);
-            // Check for special case where one of the lists is empty, since
-            // list types won't match.
-            if (firstListIter.size() == 0 || secondListIter.size() == 0) {
-                return (jaccThresh == 0.0f) ? 0.0f : -1.0f;
-            }
-            if (firstTypeTag == ATypeTag.ANY || secondTypeTag == ATypeTag.ANY)
-                throw new AlgebricksException("\n Jaccard can only be called on homogenous lists");
-            return jaccard.getSimilarity(firstListIter, secondListIter, jaccThresh);
-        }
-
-        @Override
-        protected void writeResult(float jacc) throws IOException {
-            listBuilder.reset(listType);
-            boolean matches = (jacc < 0) ? false : true;
-            inputVal.reset();
-            booleanSerde.serialize(matches ? ABoolean.TRUE : ABoolean.FALSE, inputVal.getDataOutput());
-            listBuilder.addItem(inputVal);
-
-            inputVal.reset();
-            aFloat.setValue((matches) ? jacc : 0.0f);
-            floatSerde.serialize(aFloat, inputVal.getDataOutput());
-            listBuilder.addItem(inputVal);
-
-            listBuilder.write(out, true);
-        }
-    }
-
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
index c3c6ca0..95d994f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
@@ -11,13 +11,11 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
 
-//assumes that both arguments are sorted by the same ordering
-
 public class SimilarityJaccardDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard", 2, true);
+            "similarity-jaccard", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SimilarityJaccardDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
index 4008419..24f012e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
@@ -28,7 +28,7 @@
     private static final long serialVersionUID = 1L;
 
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard-prefix-check", 6, true);
+            "similarity-jaccard-prefix-check", 6);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SimilarityJaccardPrefixCheckDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
index 4d9c528..54d4280 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
@@ -15,7 +15,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard-prefix", 6, true);
+            "similarity-jaccard-prefix", 6);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SimilarityJaccardPrefixDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
new file mode 100644
index 0000000..23b7a5d
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.common.SimilarityJaccardSortedCheckEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+// Assumes that both arguments are sorted by the same ordering.
+public class SimilarityJaccardSortedCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "similarity-jaccard-sorted-check", 3);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new SimilarityJaccardSortedCheckDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+                return new SimilarityJaccardSortedCheckEvaluator(args, output);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
new file mode 100644
index 0000000..db944d4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.common.SimilarityJaccardSortedEvaluator;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+// Assumes that both arguments are sorted by the same ordering.
+public class SimilarityJaccardSortedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "similarity-jaccard-sorted", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new SimilarityJaccardSortedDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+                return new SimilarityJaccardSortedEvaluator(args, output);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
index 644e89d..f35adee 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
@@ -28,8 +28,7 @@
 public class SpatialAreaDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-area",
-            1, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-area", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialAreaDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index 0fe63f5..c2a4d66 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -30,8 +30,7 @@
 public class SpatialCellDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-cell",
-            4, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-cell", 4);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialCellDescriptor();
@@ -75,7 +74,8 @@
                         eval3.evaluate(tuple);
 
                         try {
-                            ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput0.getByteArray()[0]);
+                            ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput0.getByteArray()[0]);
                             if (tag == ATypeTag.POINT) {
                                 double xLoc = ADoubleSerializerDeserializer.getDouble(outInput0.getByteArray(),
                                         APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
index 97e70b2..22132cc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
@@ -26,7 +26,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-distance", 2, true);
+            "spatial-distance", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialDistanceDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index be31eed..b519a23 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -37,7 +37,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-intersect", 2, true);
+            "spatial-intersect", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialIntersectDescriptor();
@@ -751,8 +751,10 @@
 
                         try {
                             boolean res = false;
-                            ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput0.getByteArray()[0]);
-                            ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput1.getByteArray()[0]);
+                            ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput0.getByteArray()[0]);
+                            ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput1.getByteArray()[0]);
 
                             switch (tag0) {
                                 case POINT:
@@ -805,12 +807,12 @@
                                             double startY1 = ADoubleSerializerDeserializer.getDouble(outInput0
                                                     .getByteArray(), ALineSerializerDeserializer
                                                     .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX1 = ADoubleSerializerDeserializer.getDouble(
-                                                    outInput0.getByteArray(), ALineSerializerDeserializer
-                                                            .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY1 = ADoubleSerializerDeserializer.getDouble(
-                                                    outInput0.getByteArray(), ALineSerializerDeserializer
-                                                            .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double endX1 = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY1 = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.Y));
 
                                             double startX2 = ADoubleSerializerDeserializer.getDouble(outInput1
                                                     .getByteArray(), ALineSerializerDeserializer
@@ -818,23 +820,26 @@
                                             double startY2 = ADoubleSerializerDeserializer.getDouble(outInput1
                                                     .getByteArray(), ALineSerializerDeserializer
                                                     .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX2 = ADoubleSerializerDeserializer.getDouble(
-                                                    outInput1.getByteArray(), ALineSerializerDeserializer
-                                                            .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY2 = ADoubleSerializerDeserializer.getDouble(
-                                                    outInput1.getByteArray(), ALineSerializerDeserializer
-                                                            .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double endX2 = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY2 = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.Y));
                                             res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2,
                                                     startY2, endX2, endY2);
                                             break;
                                         case POLYGON:
-                                            res = linePolygonIntersection(outInput0.getByteArray(), outInput1.getByteArray());
+                                            res = linePolygonIntersection(outInput0.getByteArray(),
+                                                    outInput1.getByteArray());
                                             break;
                                         case CIRCLE:
-                                            res = lineCircleIntersection(outInput0.getByteArray(), outInput1.getByteArray());
+                                            res = lineCircleIntersection(outInput0.getByteArray(),
+                                                    outInput1.getByteArray());
                                             break;
                                         case RECTANGLE:
-                                            res = lineRectangleIntersection(outInput0.getByteArray(), outInput1.getByteArray());
+                                            res = lineRectangleIntersection(outInput0.getByteArray(),
+                                                    outInput1.getByteArray());
                                             break;
                                         case NULL:
                                             res = false;
@@ -852,7 +857,8 @@
                                             res = pointInPolygon(outInput1.getByteArray(), outInput0.getByteArray());
                                             break;
                                         case LINE:
-                                            res = linePolygonIntersection(outInput1.getByteArray(), outInput0.getByteArray());
+                                            res = linePolygonIntersection(outInput1.getByteArray(),
+                                                    outInput0.getByteArray());
                                             break;
                                         case POLYGON:
                                             int numOfPoints0 = AInt16SerializerDeserializer.getShort(
@@ -880,9 +886,9 @@
                                             trianglesX1.reset();
                                             trianglesY1.reset();
                                             while (true) {
-                                                middleVertex1 = triangulatePolygon(outInput1.getByteArray(), numOfPoints1,
-                                                        pointsOffsets1, trianglesX1, trianglesY1, numOfTriangles1,
-                                                        nonSimplePolygonDetection1, middleVertex1);
+                                                middleVertex1 = triangulatePolygon(outInput1.getByteArray(),
+                                                        numOfPoints1, pointsOffsets1, trianglesX1, trianglesY1,
+                                                        numOfTriangles1, nonSimplePolygonDetection1, middleVertex1);
 
                                                 if (middleVertex1 == -1) {
                                                     break;
@@ -896,9 +902,9 @@
                                             trianglesX0.reset();
                                             trianglesY0.reset();
                                             while (true) {
-                                                middleVertex0 = triangulatePolygon(outInput0.getByteArray(), numOfPoints0,
-                                                        pointsOffsets0, trianglesX0, trianglesY0, numOfTriangles0,
-                                                        nonSimplePolygonDetection0, middleVertex0);
+                                                middleVertex0 = triangulatePolygon(outInput0.getByteArray(),
+                                                        numOfPoints0, pointsOffsets0, trianglesX0, trianglesY0,
+                                                        numOfTriangles0, nonSimplePolygonDetection0, middleVertex0);
 
                                                 if (middleVertex0 == -1) {
                                                     break;
@@ -929,7 +935,8 @@
                                             }
                                             break;
                                         case CIRCLE:
-                                            res = polygonCircleIntersection(outInput0.getByteArray(), outInput1.getByteArray());
+                                            res = polygonCircleIntersection(outInput0.getByteArray(),
+                                                    outInput1.getByteArray());
                                             break;
                                         case RECTANGLE:
                                             res = rectanglePolygonIntersection(outInput1.getByteArray(),
@@ -951,13 +958,16 @@
                                             res = pointInCircle(outInput1.getByteArray(), outInput0.getByteArray());
                                             break;
                                         case LINE:
-                                            res = lineCircleIntersection(outInput1.getByteArray(), outInput0.getByteArray());
+                                            res = lineCircleIntersection(outInput1.getByteArray(),
+                                                    outInput0.getByteArray());
                                             break;
                                         case POLYGON:
-                                            res = polygonCircleIntersection(outInput1.getByteArray(), outInput0.getByteArray());
+                                            res = polygonCircleIntersection(outInput1.getByteArray(),
+                                                    outInput0.getByteArray());
                                             break;
                                         case CIRCLE:
-                                            res = circleCircleIntersection(outInput0.getByteArray(), outInput1.getByteArray());
+                                            res = circleCircleIntersection(outInput0.getByteArray(),
+                                                    outInput1.getByteArray());
                                             break;
                                         case RECTANGLE:
                                             res = rectangleCircleIntersection(outInput1.getByteArray(),
@@ -979,7 +989,8 @@
                                             res = pointInRectangle(outInput1.getByteArray(), outInput0.getByteArray());
                                             break;
                                         case LINE:
-                                            res = lineRectangleIntersection(outInput1.getByteArray(), outInput0.getByteArray());
+                                            res = lineRectangleIntersection(outInput1.getByteArray(),
+                                                    outInput0.getByteArray());
                                             break;
                                         case POLYGON:
                                             res = rectanglePolygonIntersection(outInput0.getByteArray(),
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StartsWithDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
index 2405ad9..cd8079d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
@@ -16,8 +16,7 @@
 public class StartsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "starts-with",
-            2, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "starts-with", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StartsWithDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
new file mode 100644
index 0000000..73d2ac9
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -0,0 +1,111 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-concat",
+            1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringConcatDescriptor();
+        }
+    };
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
+    private final byte stringTypeTag = ATypeTag.STRING.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ICopyEvaluatorFactory listEvalFactory = args[0];
+                    private ArrayBackedValueStorage outInputList = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalList = listEvalFactory.createEvaluator(outInputList);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            outInputList.reset();
+                            evalList.evaluate(tuple);
+                            byte[] serOrderedList = outInputList.getByteArray();
+                            if (serOrderedList[0] == SER_NULL_TYPE_TAG) {
+                                nullSerde.serialize(ANull.NULL, out);
+                                return;
+                            }
+                            if (serOrderedList[0] != SER_ORDEREDLIST_TYPE_TAG) {
+                                throw new AlgebricksException("Expects String List."
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[0]));
+                            }
+                            int size = AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList);
+                            try {
+                                // calculate length first
+                                int utf_8_len = 0;
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    utf_8_len += UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
+                                }
+                                out.writeByte(stringTypeTag);
+                                StringUtils.writeUTF8Len(utf_8_len, out);
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    utf_8_len = UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
+                                    for (int j = 0; j < utf_8_len; j++) {
+                                        out.writeByte(serOrderedList[2 + itemOffset + j]);
+                                    }
+                                }
+                            } catch (AsterixException ex) {
+                                throw new AlgebricksException(ex);
+                            }
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(e1.getMessage());
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
new file mode 100644
index 0000000..7d5bdab
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
@@ -0,0 +1,77 @@
+/*
+ * To change this template, choose Tools | Templates
+ * and open the template in the editor.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import java.io.DataOutput;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringEndWithDescrtiptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "end-with", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringEndWithDescrtiptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1]) {
+
+                    @Override
+                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
+                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) {
+                        int len1 = UTF8StringPointable.getUTFLength(lBytes, 1);
+                        int len2 = UTF8StringPointable.getUTFLength(rBytes, 1);
+                        if (len2 > len1)
+                            return false;
+
+                        int pos = 3;
+                        int delta = len1 - len2;
+                        while (pos < len2 + 3) {
+                            char c1 = UTF8StringPointable.charAt(lBytes, pos + delta);
+                            char c2 = UTF8StringPointable.charAt(rBytes, pos);
+                            if (c1 != c2)
+                                return false;
+
+                            pos += UTF8StringPointable.charSize(lBytes, pos);
+                        }
+
+                        return true;
+                    }
+
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
new file mode 100644
index 0000000..5417e22
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -0,0 +1,73 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import java.io.DataOutput;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringEqualDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-equal",
+            2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringEqualDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1]) {
+
+                    @Override
+                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
+                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) {
+                        int len = UTF8StringPointable.getUTFLength(lBytes, 1);
+
+                        if (len != UTF8StringPointable.getUTFLength(rBytes, 1))
+                            return false;
+
+                        int pos = 3;
+                        while (pos < len + 3) {
+                            char c1 = UTF8StringPointable.charAt(lBytes, pos);
+                            char c2 = UTF8StringPointable.charAt(rBytes, pos);
+                            if (c1 != c2)
+                                return false;
+
+                            pos += UTF8StringPointable.charSize(lBytes, pos);
+                        }
+
+                        return true;
+                    }
+
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
new file mode 100644
index 0000000..83784bb
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -0,0 +1,137 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringJoinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-join", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringJoinDescriptor();
+        }
+    };
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+    private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
+    private final byte stringTypeTag = ATypeTag.STRING.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ICopyEvaluatorFactory listEvalFactory = args[0];
+                    private ICopyEvaluatorFactory sepEvalFactory = args[1];
+                    private ArrayBackedValueStorage outInputList = new ArrayBackedValueStorage();
+                    private ArrayBackedValueStorage outInputSep = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalList = listEvalFactory.createEvaluator(outInputList);
+                    private ICopyEvaluator evalSep = sepEvalFactory.createEvaluator(outInputSep);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            outInputList.reset();
+                            evalList.evaluate(tuple);
+                            byte[] serOrderedList = outInputList.getByteArray();
+
+                            outInputSep.reset();
+                            evalSep.evaluate(tuple);
+                            byte[] serSep = outInputSep.getByteArray();
+                            if (serOrderedList[0] == SER_NULL_TYPE_TAG) {
+                                nullSerde.serialize(ANull.NULL, out);
+                                return;
+                            }
+                            if (serOrderedList[0] != SER_ORDEREDLIST_TYPE_TAG) {
+                                throw new AlgebricksException("Expects String List."
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[0]));
+                            }
+
+                            if (serSep[0] == SER_NULL_TYPE_TAG) {
+                            }
+                            if (serSep[0] != SER_STRING_TYPE_TAG) {
+                                throw new AlgebricksException("Expects String as Seperator."
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[0]));
+                            }
+
+                            int size = AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList);
+                            try {
+                                // calculate length first
+                                int utf_8_len = 0;
+                                int sep_len = UTF8StringPointable.getUTFLength(serSep, 1);
+
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    int currentSize = UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
+                                    if (i != size - 1 && currentSize != 0) {
+                                        utf_8_len += sep_len;
+                                    }
+                                    utf_8_len += currentSize;
+                                }
+                                out.writeByte(stringTypeTag);
+                                StringUtils.writeUTF8Len(utf_8_len, out);
+                                for (int i = 0; i < size; i++) {
+                                    int itemOffset = AOrderedListSerializerDeserializer
+                                            .getItemOffset(serOrderedList, i);
+                                    utf_8_len = UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
+                                    for (int j = 0; j < utf_8_len; j++) {
+                                        out.writeByte(serOrderedList[2 + itemOffset + j]);
+                                    }
+                                    if (i == size - 1 || utf_8_len == 0)
+                                        continue;
+                                    for (int j = 0; j < sep_len; j++) {
+                                        out.writeByte(serSep[3 + j]);
+                                    }
+                                }
+                            } catch (AsterixException ex) {
+                                throw new AlgebricksException(ex);
+                            }
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(e1.getMessage());
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
new file mode 100644
index 0000000..1952da2
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -0,0 +1,88 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-length",
+            1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringLengthDescriptor();
+        }
+    };
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+                    private AMutableInt32 result = new AMutableInt32(0);
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(outInput);
+                    private String errorMessage = "This can not be an instance of string";
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.AINT32);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+
+                        try {
+                            outInput.reset();
+                            eval.evaluate(tuple);
+                            byte[] serString = outInput.getByteArray();
+                            if (serString[0] == SER_STRING_TYPE_TAG) {
+                                int len = UTF8StringPointable.getUTFLength(outInput.getByteArray(), 1);
+                                result.setValue(len);
+                                intSerde.serialize(result, out);
+                            } else if (serString[0] == SER_NULL_TYPE_TAG)
+                                nullSerde.serialize(ANull.NULL, out);
+                            else {
+                                throw new AlgebricksException(errorMessage);
+                            }
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(errorMessage);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
new file mode 100644
index 0000000..f576cf4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -0,0 +1,98 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lowercase", 1);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringLowerCaseDescriptor();
+        }
+    };
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(outInput);
+                    private String errorMessage = "This is not an instance of string";
+                    private final byte stt = ATypeTag.STRING.serialize();
+
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+
+                        try {
+                            outInput.reset();
+                            eval.evaluate(tuple);
+                            byte[] serString = outInput.getByteArray();
+
+                            if (serString[0] == SER_STRING_TYPE_TAG) {
+                                byte[] bytes = outInput.getByteArray();
+                                int len = UTF8StringPointable.getUTFLength(bytes, 1);
+
+                                out.writeByte(stt);
+                                StringUtils.writeUTF8Len(len, out);
+
+                                int pos = 3;
+                                while (pos < len + 3) {
+                                    char c1 = UTF8StringPointable.charAt(bytes, pos);
+                                    c1 = Character.toLowerCase(c1);
+                                    pos += UTF8StringPointable.charSize(bytes, pos);
+                                    StringUtils.writeCharAsModifiedUTF8(c1, out);
+                                }
+                            } else if (serString[0] == SER_NULL_TYPE_TAG)
+                                nullSerde.serialize(ANull.NULL, out);
+                            else
+                                throw new AlgebricksException(errorMessage);
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(errorMessage);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
new file mode 100644
index 0000000..e9e016f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
@@ -0,0 +1,112 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.utils.UTF8CharSequence;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringMatchesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "matches", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringMatchesDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1]) {
+
+                    private Pattern pattern = null;
+                    private Matcher matcher = null;
+                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
+                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
+                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private UTF8CharSequence carSeq = new UTF8CharSequence();
+
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+                    @Override
+                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
+                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) throws AlgebricksException {
+                        try {
+                            boolean newPattern = false;
+                            if (pattern == null) {
+                                newPattern = true;
+                            } else {
+                                int c = strComp.compare(rBytes, rStart, rLen, lastPattern.getByteArray(), 0,
+                                        lastPattern.size());
+                                if (c != 0) {
+                                    newPattern = true;
+                                }
+                            }
+                            if (newPattern) {
+                                lastPattern.reset();
+                                lastPattern.write(rBytes, rStart, rLen);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastPattern.getByteArray()));
+                                AString strPattern = (AString) stringSerde.deserialize(di);
+                                // pattern = Pattern.compile(toRegex(strPattern));
+                                pattern = Pattern.compile(strPattern.getStringValue());
+                            }
+
+                            carSeq.reset(array0, 1);
+                            if (newPattern) {
+                                matcher = pattern.matcher(carSeq);
+                            } else {
+                                matcher.reset(carSeq);
+                            }
+                            return matcher.find();
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
new file mode 100644
index 0000000..912a014
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
@@ -0,0 +1,139 @@
+/*
+ * To change this template, choose Tools | Templates
+ * and open the template in the editor.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.utils.UTF8CharSequence;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringMatchesWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "matches2", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringMatchesWithFlagDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractTripleStringBoolEval(dout, args[0], args[1], args[2]) {
+
+                    private Pattern pattern = null;
+                    private Matcher matcher = null;
+                    private String strPattern = "";
+                    private int flags = 0;
+                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
+                    private ByteArrayAccessibleOutputStream lastFlags = new ByteArrayAccessibleOutputStream();
+                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
+                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private UTF8CharSequence carSeq = new UTF8CharSequence();
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+                    @Override
+                    protected boolean compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2,
+                            int s2, ArrayBackedValueStorage array0, ArrayBackedValueStorage array1)
+                            throws AlgebricksException {
+                        try {
+                            boolean newPattern = false;
+                            boolean newFlags = false;
+
+                            AString astrPattern;
+                            AString astrFlags;
+
+                            if (pattern == null) {
+                                newPattern = true;
+                                newFlags = true;
+                            } else {
+                                int c = strComp.compare(b1, s1, l1, lastPattern.getByteArray(), 0, lastPattern.size());
+                                if (c != 0) {
+                                    newPattern = true;
+                                }
+
+                                c = strComp.compare(b2, s2, l2, lastFlags.getByteArray(), 0, lastFlags.size());
+                                if (c != 0) {
+                                    newFlags = true;
+                                }
+                            }
+                            if (newPattern) {
+                                lastPattern.reset();
+                                lastPattern.write(b1, s1, l1);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastPattern.getByteArray()));
+                                astrPattern = (AString) stringSerde.deserialize(di);
+                                // strPattern = toRegex(astrPattern);
+                                strPattern = astrPattern.getStringValue();
+                            }
+                            if (newFlags) {
+                                lastFlags.reset();
+                                lastFlags.write(b2, s2, l2);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastFlags.getByteArray()));
+                                astrFlags = (AString) stringSerde.deserialize(di);
+                                flags = toFlag(astrFlags);
+                            }
+
+                            pattern = Pattern.compile(strPattern, flags);
+                            carSeq.reset(array0, 1);
+                            if (newPattern) {
+                                matcher = pattern.matcher(carSeq);
+                            } else {
+                                matcher.reset(carSeq);
+                            }
+                            return matcher.find();
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
new file mode 100644
index 0000000..15c612e
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -0,0 +1,140 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.utils.UTF8CharSequence;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringReplaceDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractTripleStringStringEval(dout, args[0], args[1], args[2]) {
+
+                    private Pattern pattern = null;
+                    private Matcher matcher = null;
+                    private String replace;
+                    private String strPattern = "";
+                    private StringBuffer resultBuf = new StringBuffer();
+                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
+                    private ByteArrayAccessibleOutputStream lastReplace = new ByteArrayAccessibleOutputStream();
+                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
+                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private UTF8CharSequence carSeq = new UTF8CharSequence();
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+                    @Override
+                    protected String compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2,
+                            int s2, ArrayBackedValueStorage array0, ArrayBackedValueStorage array1)
+                            throws AlgebricksException {
+                        try {
+                            boolean newPattern = false;
+                            boolean newReplace = false;
+
+                            AString astrPattern;
+                            AString astrReplace;
+
+                            if (pattern == null) {
+                                newPattern = true;
+                                newReplace = true;
+                            } else {
+                                int c = strComp.compare(b1, s1, l1, lastPattern.getByteArray(), 0, lastPattern.size());
+                                if (c != 0) {
+                                    newPattern = true;
+                                }
+
+                                c = strComp.compare(b2, s2, l2, lastReplace.getByteArray(), 0, lastReplace.size());
+                                if (c != 0) {
+                                    newReplace = true;
+                                }
+                            }
+                            if (newPattern) {
+                                lastPattern.reset();
+                                lastPattern.write(b1, s1, l1);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastPattern.getByteArray()));
+                                astrPattern = (AString) stringSerde.deserialize(di);
+                                // strPattern = toRegex(astrPattern);
+                                strPattern = astrPattern.getStringValue();
+                            }
+                            if (newReplace) {
+                                lastReplace.reset();
+                                lastReplace.write(b2, s2, l2);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastReplace.getByteArray()));
+                                astrReplace = (AString) stringSerde.deserialize(di);
+                                replace = astrReplace.getStringValue();
+                            }
+                            if (newPattern)
+                                pattern = Pattern.compile(strPattern);
+                            carSeq.reset(array0, 1);
+                            if (newPattern) {
+                                matcher = pattern.matcher(carSeq);
+                            } else {
+                                matcher.reset(carSeq);
+                            }
+                            while (matcher.find()) {
+                                matcher.appendReplacement(resultBuf, replace);
+                            }
+                            matcher.appendTail(resultBuf);
+                            return resultBuf.toString();
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java
new file mode 100644
index 0000000..aedad20
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java
@@ -0,0 +1,163 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.utils.UTF8CharSequence;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringReplaceWithFlagsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace2", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringReplaceWithFlagsDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractQuadStringStringEval(dout, args[0], args[1], args[2], args[3]) {
+
+                    private Pattern pattern = null;
+                    private Matcher matcher = null;
+                    private String strPattern = "";
+                    private String replace = "";
+                    private int flags = 0;
+                    private StringBuffer resultBuf = new StringBuffer();
+                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
+                    private ByteArrayAccessibleOutputStream lastFlags = new ByteArrayAccessibleOutputStream();
+                    private ByteArrayAccessibleOutputStream lastReplace = new ByteArrayAccessibleOutputStream();
+                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
+                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private UTF8CharSequence carSeq = new UTF8CharSequence();
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+                    @Override
+                    protected String compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2,
+                            int s2, byte[] b3, int l3, int s3, ArrayBackedValueStorage array0,
+                            ArrayBackedValueStorage array1) throws AlgebricksException {
+                        try {
+                            boolean newPattern = false;
+                            boolean newFlags = false;
+                            boolean newReplace = false;
+
+                            AString astrPattern;
+                            AString astrFlags;
+
+                            if (pattern == null) {
+                                newPattern = true;
+                                newFlags = true;
+                            } else {
+                                int c = strComp.compare(b1, s1, l1, lastPattern.getByteArray(), 0, lastPattern.size());
+                                if (c != 0) {
+                                    newPattern = true;
+                                }
+
+                                c = strComp.compare(b3, s3, l3, lastFlags.getByteArray(), 0, lastFlags.size());
+                                if (c != 0) {
+                                    newFlags = true;
+                                }
+                            }
+
+                            if (replace == null) {
+                                newReplace = true;
+                            } else {
+                                int c = strComp.compare(b2, s2, l2, lastReplace.getByteArray(), 0, lastReplace.size());
+                                if (c != 0) {
+                                    newReplace = true;
+                                }
+                            }
+
+                            if (newPattern) {
+                                lastPattern.reset();
+                                lastPattern.write(b1, s1, l1);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastPattern.getByteArray()));
+                                astrPattern = (AString) stringSerde.deserialize(di);
+                                // strPattern = toRegex(astrPattern);
+                                strPattern = astrPattern.getStringValue();
+                            }
+                            if (newReplace) {
+                                lastReplace.reset();
+                                lastReplace.write(b2, s2, l2);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastReplace.getByteArray()));
+                                replace = ((AString) stringSerde.deserialize(di)).getStringValue();
+                            }
+                            if (newFlags) {
+                                lastFlags.reset();
+                                lastFlags.write(b3, s3, l3);
+                                // ! object creation !
+                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
+                                        lastFlags.getByteArray()));
+                                astrFlags = (AString) stringSerde.deserialize(di);
+                                flags = toFlag(astrFlags);
+                            }
+
+                            if (newPattern || newFlags)
+                                pattern = Pattern.compile(strPattern, flags);
+                            resultBuf.setLength(0);
+                            carSeq.reset(array0, 1);
+                            if (newPattern) {
+                                matcher = pattern.matcher(carSeq);
+                            } else {
+                                matcher.reset(carSeq);
+                            }
+                            while (matcher.find()) {
+                                matcher.appendReplacement(resultBuf, replace);
+                            }
+                            matcher.appendTail(resultBuf);
+                            return resultBuf.toString();
+                        } catch (HyracksDataException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java
new file mode 100644
index 0000000..9ecdee9
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringStartWithDescrtiptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "start-with", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringStartWithDescrtiptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1]) {
+
+                    @Override
+                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
+                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) {
+                        int patternLength = UTF8StringPointable.getUTFLength(rBytes, 1);
+                        if (patternLength > UTF8StringPointable.getUTFLength(lBytes, 1))
+                            return false;
+
+                        int pos = 3;
+                        while (pos < patternLength + 3) {
+                            char c1 = UTF8StringPointable.charAt(lBytes, pos);
+                            char c2 = UTF8StringPointable.charAt(rBytes, pos);
+                            if (c1 != c2)
+                                return false;
+
+                            pos += UTF8StringPointable.charSize(lBytes, pos);
+                        }
+
+                        return true;
+                    }
+
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
new file mode 100644
index 0000000..3c60e28
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -0,0 +1,147 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.builders.IAOrderedListBuilder;
+import edu.uci.ics.asterix.builders.OrderedListBuilder;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * @author Xiaoyu Ma
+ */
+public class StringToCodePointDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-to-codepoint", 1);
+    public static final IFunctionDescriptorFactory FACTORY1 = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringToCodePointDescriptor();
+        }
+    };
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringToCodePointDescriptor();
+        }
+    };
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+                    protected final DataOutput out = output.getDataOutput();;
+                    protected final ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    protected final ICopyEvaluator stringEval = args[0].createEvaluator(argOut);
+                    protected final AOrderedListType intListType = new AOrderedListType(BuiltinType.AINT32, null);
+
+                    private IAOrderedListBuilder listBuilder = new OrderedListBuilder();
+                    private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
+
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.AINT32);
+                    private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+                    int UTF8ToCodePoint(byte[] b, int s) {
+                        if (b[s] >> 7 == 0) {
+                            // 1 byte
+                            return b[s];
+                        } else if ((b[s] & 0xe0) == 0xc0) { /*0xe0 = 0b1110000*/
+                            // 2 bytes
+                            return ((int) (b[s] & 0x1f)) << 6 | /*0x3f = 0b00111111*/
+                            ((int) (b[s + 1] & 0x3f));
+                        } else if ((b[s] & 0xf0) == 0xe0) {
+                            // 3bytes
+                            return ((int) (b[s] & 0xf)) << 12 | ((int) (b[s + 1] & 0x3f)) << 6
+                                    | ((int) (b[s + 2] & 0x3f));
+                        } else if ((b[s] & 0xf8) == 0xf0) {
+                            // 4bytes
+                            return ((int) (b[s] & 0x7)) << 18 | ((int) (b[s + 1] & 0x3f)) << 12
+                                    | ((int) (b[s + 2] & 0x3f)) << 6 | ((int) (b[s + 3] & 0x3f));
+                        } else if ((b[s] & 0xfc) == 0xf8) {
+                            // 5bytes
+                            return ((int) (b[s] & 0x3)) << 24 | ((int) (b[s + 1] & 0x3f)) << 18
+                                    | ((int) (b[s + 2] & 0x3f)) << 12 | ((int) (b[s + 3] & 0x3f)) << 6
+                                    | ((int) (b[s + 4] & 0x3f));
+                        } else if ((b[s] & 0xfe) == 0xfc) {
+                            // 6bytes
+                            return ((int) (b[s] & 0x1)) << 30 | ((int) (b[s + 1] & 0x3f)) << 24
+                                    | ((int) (b[s + 2] & 0x3f)) << 18 | ((int) (b[s + 3] & 0x3f)) << 12
+                                    | ((int) (b[s + 4] & 0x3f)) << 6 | ((int) (b[s + 5] & 0x3f));
+                        }
+                        return 0;
+                    }
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            argOut.reset();
+                            stringEval.evaluate(tuple);
+                            byte[] serString = argOut.getByteArray();
+
+                            if (serString[0] == SER_STRING_TYPE_TAG) {
+                                byte[] bytes = argOut.getByteArray();
+                                int len = UTF8StringPointable.getUTFLength(bytes, 1);
+
+                                int pos = 3;
+                                listBuilder.reset(intListType);
+                                while (pos < len + 3) {
+                                    int codePoint = UTF8ToCodePoint(bytes, pos);
+                                    pos += UTF8StringPointable.charSize(bytes, pos);
+
+                                    inputVal.reset();
+                                    aInt32.setValue(codePoint);
+                                    int32Serde.serialize(aInt32, inputVal.getDataOutput());
+                                    listBuilder.addItem(inputVal);
+
+                                }
+                                listBuilder.write(out, true);
+                            } else if (serString[0] == SER_NULL_TYPE_TAG)
+                                nullSerde.serialize(ANull.NULL, out);
+                            else
+                                throw new AlgebricksException("Expects String Type.");
+                        } catch (IOException e1) {
+                            throw new AlgebricksException(e1.getMessage());
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/Substring2Descriptor.java
new file mode 100644
index 0000000..ebbfe80
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -0,0 +1,91 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class Substring2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "substring2", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new Substring2Descriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalString = args[0].createEvaluator(argOut);
+                    private ICopyEvaluator evalStart = args[1].createEvaluator(argOut);
+                    private final byte stt = ATypeTag.STRING.serialize();
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        argOut.reset();
+                        evalStart.evaluate(tuple);
+                        int start = IntegerSerializerDeserializer.getInt(argOut.getByteArray(), 1) - 1;
+                        argOut.reset();
+                        evalString.evaluate(tuple);
+
+                        byte[] bytes = argOut.getByteArray();
+                        int utflen = UTF8StringPointable.getUTFLength(bytes, 1);
+                        int sStart = 3;
+                        int c = 0;
+                        int idxPos1 = 0;
+                        // skip to start
+                        while (idxPos1 < start && c < utflen) {
+                            c += UTF8StringPointable.charSize(bytes, sStart + c);
+                            ++idxPos1;
+                        }
+                        int startSubstr = c;
+
+                        while (c < utflen) {
+                            c += UTF8StringPointable.charSize(bytes, sStart + c);
+                        }
+
+                        int substrByteLen = c - startSubstr;
+                        try {
+                            out.writeByte(stt);
+                            out.writeByte((byte) ((substrByteLen >>> 8) & 0xFF));
+                            out.writeByte((byte) ((substrByteLen >>> 0) & 0xFF));
+                            out.write(bytes, sStart + startSubstr, substrByteLen);
+
+                        } catch (IOException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
new file mode 100644
index 0000000..567526d
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
@@ -0,0 +1,101 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class SubstringAfterDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "substring-after", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new SubstringAfterDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+                    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalString = args[0].createEvaluator(array0);
+                    private ICopyEvaluator evalPattern = args[1].createEvaluator(array1);
+                    private final byte stt = ATypeTag.STRING.serialize();
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        array0.reset();
+                        evalString.evaluate(tuple);
+                        byte[] src = array0.getByteArray();
+
+                        array1.reset();
+                        evalPattern.evaluate(tuple);
+                        byte[] pattern = array1.getByteArray();
+
+                        int srcLen = UTF8StringPointable.getUTFLength(src, 1);
+                        int patternLen = UTF8StringPointable.getUTFLength(pattern, 1);
+                        int posSrc = 3;
+                        int posPattern = 3;
+
+                        int offset = 0;
+                        // boolean found = false;
+                        while (posSrc - 3 < srcLen - patternLen) {
+                            offset = 0;
+                            while (posPattern + offset - 3 < patternLen && posSrc + offset - 3 < srcLen) {
+                                char c1 = UTF8StringPointable.charAt(src, posSrc + offset);
+                                char c2 = UTF8StringPointable.charAt(pattern, posPattern + offset);
+                                if (c1 != c2)
+                                    break;
+                                offset++;
+                            }
+                            if (offset == patternLen) {
+                                // found = true;
+                                break;
+                            }
+                            posSrc += UTF8StringPointable.charSize(src, posSrc);
+                        }
+
+                        posSrc += patternLen;
+                        int substrByteLen = srcLen - posSrc + 3;
+                        try {
+                            out.writeByte(stt);
+                            out.writeByte((byte) ((substrByteLen >>> 8) & 0xFF));
+                            out.writeByte((byte) ((substrByteLen >>> 0) & 0xFF));
+                            out.write(src, posSrc, substrByteLen);
+
+                        } catch (IOException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
new file mode 100644
index 0000000..fb496a1
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -0,0 +1,101 @@
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class SubstringBeforeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "substring-before", 2);
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new SubstringBeforeDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+                    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
+                    private ArrayBackedValueStorage array1 = new ArrayBackedValueStorage();
+                    private ICopyEvaluator evalString = args[0].createEvaluator(array0);
+                    private ICopyEvaluator evalPattern = args[1].createEvaluator(array1);
+                    private final byte stt = ATypeTag.STRING.serialize();
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        array0.reset();
+                        evalString.evaluate(tuple);
+                        byte[] src = array0.getByteArray();
+
+                        array1.reset();
+                        evalPattern.evaluate(tuple);
+                        byte[] pattern = array1.getByteArray();
+
+                        int srcLen = UTF8StringPointable.getUTFLength(src, 1);
+                        int patternLen = UTF8StringPointable.getUTFLength(pattern, 1);
+                        int posSrc = 3;
+                        int posPattern = 3;
+
+                        int offset = 0;
+                        while (posSrc - 3 < srcLen - patternLen) {
+                            // offset = 0;                        
+                            while (posPattern + offset - 3 < patternLen && posSrc + offset - 3 < srcLen) {
+                                char c1 = UTF8StringPointable.charAt(src, posSrc + offset);
+                                char c2 = UTF8StringPointable.charAt(pattern, posPattern + offset);
+                                if (c1 != c2)
+                                    break;
+                                offset++;
+                            }
+                            if (offset == patternLen) {
+                                // found = true;
+                                break;
+                            }
+                            posSrc += UTF8StringPointable.charSize(src, posSrc);
+                        }
+                        int startSubstr = 3;
+
+                        int substrByteLen = posSrc - startSubstr;
+                        try {
+                            out.writeByte(stt);
+                            out.writeByte((byte) ((substrByteLen >>> 8) & 0xFF));
+                            out.writeByte((byte) ((substrByteLen >>> 0) & 0xFF));
+                            out.write(src, startSubstr, substrByteLen);
+
+                        } catch (IOException e) {
+                            throw new AlgebricksException(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index c6d0627..5d3007f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -21,8 +21,7 @@
 public class SubstringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "substring", 3,
-            true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "substring", 3);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubstringDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
index 29fbb7f..4a33010 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
@@ -19,7 +19,7 @@
 
     private static final long serialVersionUID = 1L;
     public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "switch-case",
-            FunctionIdentifier.VARARGS, true);
+            FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SwitchCaseDescriptor();
@@ -79,7 +79,8 @@
                             // the default case
                             argOut.reset();
                             evals[n - 1].evaluate(tuple);
-                            output.getDataOutput().write(argOut.getByteArray(), argOut.getStartOffset(), argOut.getLength());
+                            output.getDataOutput().write(argOut.getByteArray(), argOut.getStartOffset(),
+                                    argOut.getLength());
                         } catch (HyracksDataException hde) {
                             throw new AlgebricksException(hde);
                         } catch (IOException ioe) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
index ea0765e..39149ac 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/UnorderedListConstructorDescriptor.java
@@ -22,7 +22,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "unordered-list-constructor", FunctionIdentifier.VARARGS, true);
+            "unordered-list-constructor", FunctionIdentifier.VARARGS);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new UnorderedListConstructorDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
index 75a211f..8f8c5f3 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
@@ -7,21 +7,20 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.common.WordTokensEvaluator;
-import edu.uci.ics.fuzzyjoin.tokenizer.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.IBinaryTokenizer;
-import edu.uci.ics.fuzzyjoin.tokenizer.ITokenFactory;
-import edu.uci.ics.fuzzyjoin.tokenizer.UTF8WordTokenFactory;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
 
 public class WordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "word-tokens",
-            1, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "word-tokens", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new WordTokensDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/YearDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/YearDescriptor.java
index b95fbfa..e6da3ac 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/YearDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/YearDescriptor.java
@@ -27,7 +27,7 @@
 public class YearDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year", 1, true);
+    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year", 1);
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index e16b384..e9f1ac5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -91,6 +91,8 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.CreateRectangleDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.EditDistanceCheckDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.EditDistanceDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.EditDistanceListIsFilterable;
+import edu.uci.ics.asterix.runtime.evaluators.functions.EditDistanceStringIsFilterable;
 import edu.uci.ics.asterix.runtime.evaluators.functions.EmbedTypeDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.EndsWithDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.FieldAccessByIndexDescriptor;
@@ -107,6 +109,7 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericAddDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericSubtractDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
@@ -119,6 +122,8 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.SimilarityJaccardDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixCheckDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SimilarityJaccardSortedCheckDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SimilarityJaccardSortedDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
@@ -129,6 +134,28 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.YearDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringEqualDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringStartWithDescrtiptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringEndWithDescrtiptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringMatchesDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringLowerCaseDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringMatchesWithFlagDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringReplaceWithFlagsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringLengthDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.Substring2Descriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringConcatDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.StringJoinDescriptor;
 import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
 import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
 import edu.uci.ics.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
@@ -151,10 +178,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
-import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
@@ -234,12 +261,38 @@
         temp.add(NumericDivideDescriptor.FACTORY);
         temp.add(NumericMultiplyDescriptor.FACTORY);
         temp.add(NumericSubtractDescriptor.FACTORY);
+        temp.add(NumericModuloDescriptor.FACTORY);
         temp.add(IsNullDescriptor.FACTORY);
         temp.add(NotDescriptor.FACTORY);
         temp.add(LenDescriptor.FACTORY);
         temp.add(NonEmptyStreamAggregateDescriptor.FACTORY);
         temp.add(RangeDescriptor.FACTORY);
 
+// Xiaoyu Ma add for numeric unary functions
+        temp.add(NumericAbsDescriptor.FACTORY);
+        temp.add(NumericCeilingDescriptor.FACTORY);
+        temp.add(NumericFloorDescriptor.FACTORY);
+        temp.add(NumericRoundDescriptor.FACTORY);
+        temp.add(NumericRoundHalfToEvenDescriptor.FACTORY);
+        temp.add(NumericRoundHalfToEven2Descriptor.FACTORY);
+        // String functions
+        temp.add(StringEqualDescriptor.FACTORY);
+        temp.add(StringStartWithDescrtiptor.FACTORY);    
+        temp.add(StringEndWithDescrtiptor.FACTORY);       
+        temp.add(StringMatchesDescriptor.FACTORY);    
+        temp.add(StringLowerCaseDescriptor.FACTORY);   
+        temp.add(StringMatchesWithFlagDescriptor.FACTORY);
+        temp.add(StringReplaceDescriptor.FACTORY);      
+        temp.add(StringReplaceWithFlagsDescriptor.FACTORY);    
+        temp.add(StringLengthDescriptor.FACTORY);        
+        temp.add(Substring2Descriptor.FACTORY);    
+        temp.add(SubstringBeforeDescriptor.FACTORY); 
+        temp.add(SubstringAfterDescriptor.FACTORY); 
+        temp.add(StringToCodePointDescriptor.FACTORY);         
+        temp.add(CodePointToStringDescriptor.FACTORY); 
+        temp.add(StringConcatDescriptor.FACTORY);         
+        temp.add(StringJoinDescriptor.FACTORY);      
+
         // aggregates
         temp.add(ListifyAggregateDescriptor.FACTORY);
         temp.add(CountAggregateDescriptor.FACTORY);
@@ -305,9 +358,13 @@
 
         temp.add(EditDistanceDescriptor.FACTORY);
         temp.add(EditDistanceCheckDescriptor.FACTORY);
+        temp.add(EditDistanceStringIsFilterable.FACTORY);
+        temp.add(EditDistanceListIsFilterable.FACTORY);
 
         temp.add(SimilarityJaccardDescriptor.FACTORY);
         temp.add(SimilarityJaccardCheckDescriptor.FACTORY);
+        temp.add(SimilarityJaccardSortedDescriptor.FACTORY);
+        temp.add(SimilarityJaccardSortedCheckDescriptor.FACTORY);
         temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
         temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
 
@@ -324,8 +381,8 @@
     }
 
     @Override
-    public IBinaryBooleanInspector getBinaryBooleanInspector() {
-        return AqlBinaryBooleanInspectorImpl.INSTANCE;
+    public IBinaryBooleanInspectorFactory getBinaryBooleanInspectorFactory() {
+        return AqlBinaryBooleanInspectorImpl.FACTORY;
     }
 
     @Override
@@ -442,8 +499,8 @@
                         new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
                         new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
                                 new AInt32(i)))));
-                return new Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory, partitionFun,
-                        recType.getFieldTypes()[i]);
+                return new Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>(evalFactory,
+                        partitionFun, recType.getFieldTypes()[i]);
             }
         }
         throw new AlgebricksException("Could not find field " + fldName + " in the schema.");
@@ -573,8 +630,8 @@
     }
 
     @Override
-    public IBinaryIntegerInspector getBinaryIntegerInspector() {
-        return AqlBinaryIntegerInspector.INSTANCE;
+    public IBinaryIntegerInspectorFactory getBinaryIntegerInspectorFactory() {
+        return AqlBinaryIntegerInspector.FACTORY;
     }
 
     @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/std/NoTupleSourceRuntimeFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/std/NoTupleSourceRuntimeFactory.java
index 4038572..ee432f7 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/std/NoTupleSourceRuntimeFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/std/NoTupleSourceRuntimeFactory.java
@@ -16,8 +16,8 @@
 
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.context.RuntimeContext;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputSourcePushRuntime;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class NoTupleSourceRuntimeFactory implements IPushRuntimeFactory {
@@ -30,7 +30,7 @@
     }
 
     @Override
-    public IPushRuntime createPushRuntime(final RuntimeContext context) {
+    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) {
         return new AbstractOneInputSourcePushRuntime() {
 
             @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 69e823b..f4bc68d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -23,13 +23,13 @@
 public class TidRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new TidRunningAggregateDescriptor();
         }
     };
-    
+
     @Override
     public ICopyRunningAggregateFunctionFactory createRunningAggregateFunctionFactory(ICopyEvaluatorFactory[] args)
             throws AlgebricksException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
index fb7f8f0..03d2677 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
@@ -26,13 +26,13 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2, true);
+    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RangeDescriptor();
         }
     };
-    
+
     @Override
     public FunctionIdentifier getIdentifier() {
         return FID;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index bfd23eb..cfac382 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -31,7 +31,7 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "scan-collection", 1, true);
+            "scan-collection", 1);
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScanCollectionDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
index cd69f5b..8aee5cf 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
@@ -32,12 +32,12 @@
 
     private static final long serialVersionUID = 1L;
     private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subset-collection", 3, true);
+            "subset-collection", 3);
 
     private final static byte SER_ORDEREDLIST_TYPE_TAG = ATypeTag.ORDEREDLIST.serialize();
     private final static byte SER_UNORDEREDLIST_TYPE_TAG = ATypeTag.UNORDEREDLIST.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
-    
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubsetCollectionDescriptor();
@@ -51,7 +51,8 @@
             private static final long serialVersionUID = 1L;
 
             @Override
-            public ICopyUnnestingFunction createUnnestingFunction(IDataOutputProvider provider) throws AlgebricksException {
+            public ICopyUnnestingFunction createUnnestingFunction(IDataOutputProvider provider)
+                    throws AlgebricksException {
 
                 final DataOutput out = provider.getDataOutput();
 
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/CustOrdDataGen.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/CustOrdDataGen.java
index 29c5330..da6c419 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/CustOrdDataGen.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/CustOrdDataGen.java
@@ -33,7 +33,8 @@
     private static final int MAX_INTERESTS = 5;
     private String[] INTERESTS = { "Bass", "Music", "Databases", "Fishing", "Tennis", "Squash", "Computers", "Books",
             "Movies", "Cigars", "Wine", "Running", "Walking", "Skiing", "Basketball", "Video Games", "Cooking",
-            "Coffee", "Base Jumping", "Puzzles" };
+            "Coffee", "Base Jumping", "Puzzles", "Chess", "Programming", "Reddit", "Soccer", "Hockey", "Money",
+            "Dancing", "Brewing", "Gardening", "Hacking", "Reading"};
 
     private static final int MIN_CHILD_AGE = 0;