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;